diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index 1c0d8934a20..4b7ca00664f 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -65,7 +65,6 @@ import ( "github.com/onflow/flow-go/state/protocol/blocktimer" "github.com/onflow/flow-go/state/protocol/events/gadgets" protocol_state "github.com/onflow/flow-go/state/protocol/protocol_state/state" - "github.com/onflow/flow-go/storage" bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/utils/io" ) @@ -104,32 +103,31 @@ func main() { insecureAccessAPI bool accessNodeIDS []string - err error - mutableState protocol.ParticipantState - beaconPrivateKey *encodable.RandomBeaconPrivKey - guarantees mempool.Guarantees - receipts mempool.ExecutionTree - seals mempool.IncorporatedResultSeals - pendingReceipts mempool.PendingReceipts - receiptRequester *requester.Engine - syncCore *chainsync.Core - comp *compliance.Engine - hot module.HotStuff - conMetrics module.ConsensusMetrics - machineAccountMetrics module.MachineAccountMetrics - mainMetrics module.HotstuffMetrics - receiptValidator module.ReceiptValidator - chunkAssigner *chmodule.ChunkAssigner - followerDistributor *pubsub.FollowerDistributor - dkgBrokerTunnel *dkgmodule.BrokerTunnel - blockTimer protocol.BlockTimer - proposalDurProvider hotstuff.ProposalDurationProvider - committee *committees.Consensus - epochLookup *epochs.EpochLookup - hotstuffModules *consensus.HotstuffModules - dkgState *bstorage.DKGState - safeBeaconKeys *bstorage.SafeBeaconPrivateKeys - getSealingConfigs module.SealingConfigsGetter + err error + mutableState protocol.ParticipantState + beaconPrivateKey *encodable.RandomBeaconPrivKey + guarantees mempool.Guarantees + receipts mempool.ExecutionTree + seals mempool.IncorporatedResultSeals + pendingReceipts mempool.PendingReceipts + receiptRequester *requester.Engine + syncCore *chainsync.Core + comp *compliance.Engine + hot module.HotStuff + conMetrics module.ConsensusMetrics + machineAccountMetrics module.MachineAccountMetrics + mainMetrics module.HotstuffMetrics + receiptValidator module.ReceiptValidator + chunkAssigner *chmodule.ChunkAssigner + followerDistributor *pubsub.FollowerDistributor + dkgBrokerTunnel *dkgmodule.BrokerTunnel + blockTimer protocol.BlockTimer + proposalDurProvider hotstuff.ProposalDurationProvider + committee *committees.Consensus + epochLookup *epochs.EpochLookup + hotstuffModules *consensus.HotstuffModules + myBeaconKeyStateMachine *bstorage.RecoverablePrivateBeaconKeyStateMachine + getSealingConfigs module.SealingConfigsGetter ) var deprecatedFlagBlockRateDelay time.Duration @@ -214,13 +212,9 @@ func main() { return nil }). Module("dkg state", func(node *cmd.NodeConfig) error { - dkgState, err = bstorage.NewDKGState(node.Metrics.Cache, node.SecretsDB) + myBeaconKeyStateMachine, err = bstorage.NewRecoverableRandomBeaconStateMachine(node.Metrics.Cache, node.SecretsDB) return err }). - Module("beacon keys", func(node *cmd.NodeConfig) error { - safeBeaconKeys = bstorage.NewSafeBeaconPrivateKeys(dkgState) - return nil - }). Module("updatable sealing config", func(node *cmd.NodeConfig) error { setter, err := updatable_configs.NewSealingConfigs( requiredApprovalsForSealConstruction, @@ -344,22 +338,24 @@ func main() { myBeaconPublicKeyShare) } - // store my beacon key for the first epoch post-spork - err = dkgState.InsertMyBeaconPrivateKey(epochCounter, beaconPrivateKey.PrivateKey) - if err != nil && !errors.Is(err, storage.ErrAlreadyExists) { - return err + started, err := myBeaconKeyStateMachine.GetDKGStarted(epochCounter) + if err != nil { + return fmt.Errorf("could not get DKG started flag for root epoch %d: %w", epochCounter, err) } - // mark the root DKG as successful, so it is considered safe to use the key - err = dkgState.SetDKGEndState(epochCounter, flow.DKGEndStateSuccess) - if err != nil && !errors.Is(err, storage.ErrAlreadyExists) { - return err + + // perform this only if state machine is in initial state + if !started { + // store my beacon key for the first epoch post-spork + err = myBeaconKeyStateMachine.UpsertMyBeaconPrivateKey(epochCounter, beaconPrivateKey.PrivateKey) + if err != nil { + return fmt.Errorf("could not upsert my beacon private key for root epoch %d: %w", epochCounter, err) + } } return nil }). Module("my beacon key epoch recovery", func(node *cmd.NodeConfig) error { - recoverMyBeaconKeyStorage := bstorage.NewEpochRecoveryMyBeaconKey(safeBeaconKeys) - myBeaconKeyRecovery, err := dkgmodule.NewBeaconKeyRecovery(node.Logger, node.Me, node.State, recoverMyBeaconKeyStorage) + myBeaconKeyRecovery, err := dkgmodule.NewBeaconKeyRecovery(node.Logger, node.Me, node.State, myBeaconKeyStateMachine) if err != nil { return fmt.Errorf("could not initialize my beacon key epoch recovery: %w", err) } @@ -582,7 +578,7 @@ func main() { // wrap Main consensus committee with metrics wrappedCommittee := committees.NewMetricsWrapper(committee, mainMetrics) // wrapper for measuring time spent determining consensus committee relations - beaconKeyStore := hotsignature.NewEpochAwareRandomBeaconKeyStore(epochLookup, safeBeaconKeys) + beaconKeyStore := hotsignature.NewEpochAwareRandomBeaconKeyStore(epochLookup, myBeaconKeyStateMachine) // initialize the combined signer for hotstuff var signer hotstuff.Signer @@ -922,7 +918,7 @@ func main() { node.Logger, node.Me, node.State, - dkgState, + myBeaconKeyStateMachine, dkgmodule.NewControllerFactory( node.Logger, node.Me, diff --git a/cmd/util/cmd/common/node_info.go b/cmd/util/cmd/common/node_info.go index 061741d0955..0b5093954f5 100644 --- a/cmd/util/cmd/common/node_info.go +++ b/cmd/util/cmd/common/node_info.go @@ -41,16 +41,16 @@ func ReadFullPartnerNodeInfos(log zerolog.Logger, partnerWeightsPath, partnerNod } err = ValidateNetworkPubKey(partner.NetworkPubKey) if err != nil { - return nil, fmt.Errorf(fmt.Sprintf("invalid network public key: %s", partner.NetworkPubKey)) + return nil, fmt.Errorf("invalid network public key: %s", partner.NetworkPubKey) } err = ValidateStakingPubKey(partner.StakingPubKey) if err != nil { - return nil, fmt.Errorf(fmt.Sprintf("invalid staking public key: %s", partner.StakingPubKey)) + return nil, fmt.Errorf("invalid staking public key: %s", partner.StakingPubKey) } weight := weights[partner.NodeID] if valid := ValidateWeight(weight); !valid { - return nil, fmt.Errorf(fmt.Sprintf("invalid partner weight: %d", weight)) + return nil, fmt.Errorf("invalid partner weight: %d", weight) } if weight != flow.DefaultInitialWeight { diff --git a/engine/common/grpc/forwarder/forwarder.go b/engine/common/grpc/forwarder/forwarder.go index a0af264b55a..3b9b44d269e 100644 --- a/engine/common/grpc/forwarder/forwarder.go +++ b/engine/common/grpc/forwarder/forwarder.go @@ -75,7 +75,7 @@ func (f *Forwarder) reconnectingClient(i int) error { // FaultTolerantClient implements an upstream connection that reconnects on errors // a reasonable amount of time. func (f *Forwarder) FaultTolerantClient() (access.AccessAPIClient, io.Closer, error) { - if f.upstream == nil || len(f.upstream) == 0 { + if len(f.upstream) == 0 { return nil, nil, status.Errorf(codes.Unimplemented, "method not implemented") } diff --git a/engine/consensus/dkg/reactor_engine.go b/engine/consensus/dkg/reactor_engine.go index daedd8bac56..c60c815bce4 100644 --- a/engine/consensus/dkg/reactor_engine.go +++ b/engine/consensus/dkg/reactor_engine.go @@ -154,12 +154,12 @@ func (e *ReactorEngine) startDKGForEpoch(currentEpochCounter uint64, first *flow Hex("first_block_id", firstID[:]). // id of first block in EpochSetup phase Logger() - // if we have started the dkg for this epoch already, exit + // if we have dkgState the dkg for this epoch already, exit started, err := e.dkgState.GetDKGStarted(nextEpochCounter) if err != nil { // unexpected storage-level error // TODO use irrecoverable context - log.Fatal().Err(err).Msg("could not check whether DKG is started") + log.Fatal().Err(err).Msg("could not check whether DKG is dkgState") } if started { log.Warn().Msg("DKG started before, skipping starting the DKG for this epoch") @@ -167,11 +167,11 @@ func (e *ReactorEngine) startDKGForEpoch(currentEpochCounter uint64, first *flow } // flag that we are starting the dkg for this epoch - err = e.dkgState.SetDKGStarted(nextEpochCounter) + err = e.dkgState.SetDKGState(nextEpochCounter, flow.DKGStateStarted) if err != nil { // unexpected storage-level error // TODO use irrecoverable context - log.Fatal().Err(err).Msg("could not set dkg started") + log.Fatal().Err(err).Msg("could not set dkg dkgState") } curDKGInfo, err := e.getDKGInfo(firstID) @@ -271,9 +271,13 @@ func (e *ReactorEngine) handleEpochCommittedPhaseStarted(currentEpochCounter uin // This can happen if the DKG failed locally, if we failed to generate // a local private beacon key, or if we crashed while performing this // check previously. - endState, err := e.dkgState.GetDKGEndState(nextEpochCounter) - if err == nil { - log.Warn().Msgf("checking beacon key consistency: exiting because dkg end state was already set: %s", endState.String()) + currentState, err := e.dkgState.GetDKGState(nextEpochCounter) + if err != nil { + log.Fatal().Err(err).Msg("failed to get dkg state, by this point it should have been set") + return + } + if currentState != flow.DKGStateCompleted { + log.Warn().Msgf("checking beacon key consistency: exiting because dkg didn't reach completed state: %s", currentState.String()) return } @@ -289,10 +293,10 @@ func (e *ReactorEngine) handleEpochCommittedPhaseStarted(currentEpochCounter uin return } - myBeaconPrivKey, err := e.dkgState.RetrieveMyBeaconPrivateKey(nextEpochCounter) + myBeaconPrivKey, err := e.dkgState.UnsafeRetrieveMyBeaconPrivateKey(nextEpochCounter) if errors.Is(err, storage.ErrNotFound) { log.Warn().Msg("checking beacon key consistency: no key found") - err := e.dkgState.SetDKGEndState(nextEpochCounter, flow.DKGEndStateNoKey) + err := e.dkgState.SetDKGState(nextEpochCounter, flow.DKGStateFailure) if err != nil { // TODO use irrecoverable context log.Fatal().Err(err).Msg("failed to set dkg end state") @@ -319,7 +323,7 @@ func (e *ReactorEngine) handleEpochCommittedPhaseStarted(currentEpochCounter uin Str("computed_beacon_pub_key", localPubKey.String()). Str("canonical_beacon_pub_key", nextDKGPubKey.String()). Msg("checking beacon key consistency: locally computed beacon public key does not match beacon public key for next epoch") - err := e.dkgState.SetDKGEndState(nextEpochCounter, flow.DKGEndStateInconsistentKey) + err := e.dkgState.SetDKGState(nextEpochCounter, flow.DKGStateFailure) if err != nil { // TODO use irrecoverable context log.Fatal().Err(err).Msg("failed to set dkg end state") @@ -327,7 +331,7 @@ func (e *ReactorEngine) handleEpochCommittedPhaseStarted(currentEpochCounter uin return } - err = e.dkgState.SetDKGEndState(nextEpochCounter, flow.DKGEndStateSuccess) + err = e.dkgState.SetDKGState(nextEpochCounter, flow.RandomBeaconKeyCommitted) if err != nil { // TODO use irrecoverable context e.log.Fatal().Err(err).Msg("failed to set dkg end state") @@ -427,10 +431,10 @@ func (e *ReactorEngine) end(nextEpochCounter uint64) func() error { // has already abandoned the happy path, because on the happy path the ReactorEngine is the only writer. // Then this function just stops and returns without error. e.log.Warn().Err(err).Msgf("node %s with index %d failed DKG locally", e.me.NodeID(), e.controller.GetIndex()) - err := e.dkgState.SetDKGEndState(nextEpochCounter, flow.DKGEndStateDKGFailure) + err := e.dkgState.SetDKGState(nextEpochCounter, flow.DKGStateFailure) if err != nil { if errors.Is(err, storage.ErrAlreadyExists) { - return nil // DKGEndState already being set is expected in case of epoch recovery + return nil // DKGState already being set is expected in case of epoch recovery } return fmt.Errorf("failed to set dkg end state following dkg end error: %w", err) } diff --git a/engine/consensus/dkg/reactor_engine_test.go b/engine/consensus/dkg/reactor_engine_test.go index 7a0a1917c4c..cdf4a853616 100644 --- a/engine/consensus/dkg/reactor_engine_test.go +++ b/engine/consensus/dkg/reactor_engine_test.go @@ -139,7 +139,7 @@ func (suite *ReactorEngineSuite_SetupPhase) SetupTest() { // ensure that an attempt is made to insert the expected dkg private share // for the next epoch. suite.dkgState = new(storage.DKGState) - suite.dkgState.On("SetDKGStarted", suite.NextEpochCounter()).Return(nil).Once() + suite.dkgState.On("SetDKGState", suite.NextEpochCounter(), flow.DKGStateStarted).Return(nil).Once() suite.dkgState.On("InsertMyBeaconPrivateKey", mock.Anything, mock.Anything).Run( func(args mock.Arguments) { epochCounter := args.Get(0).(uint64) @@ -266,7 +266,7 @@ type ReactorEngineSuite_CommittedPhase struct { epochCounter uint64 // current epoch counter myLocalBeaconKey crypto.PrivateKey // my locally computed beacon key myGlobalBeaconPubKey crypto.PublicKey // my public key, as dictated by global DKG - dkgEndState flow.DKGEndState // backend for DGKState. + DKGState flow.DKGState // backend for DGKState. firstBlock *flow.Header // first block of EpochCommitted phase warnsLogged int // count # of warn-level logs @@ -290,7 +290,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) NextEpochCounter() uint64 { func (suite *ReactorEngineSuite_CommittedPhase) SetupTest() { suite.epochCounter = rand.Uint64() - suite.dkgEndState = flow.DKGEndStateUnknown + suite.DKGState = flow.DKGStateCompleted // we start with the completed state since we are going to test the transition to committed suite.me = new(module.Local) id := unittest.IdentifierFixture() @@ -301,7 +301,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) SetupTest() { suite.myGlobalBeaconPubKey = suite.myLocalBeaconKey.PublicKey() suite.dkgState = new(storage.DKGState) - suite.dkgState.On("RetrieveMyBeaconPrivateKey", suite.NextEpochCounter()).Return( + suite.dkgState.On("UnsafeRetrieveMyBeaconPrivateKey", suite.NextEpochCounter()).Return( func(_ uint64) crypto.PrivateKey { return suite.myLocalBeaconKey }, func(_ uint64) error { if suite.myLocalBeaconKey == nil { @@ -310,17 +310,17 @@ func (suite *ReactorEngineSuite_CommittedPhase) SetupTest() { return nil }, ) - suite.dkgState.On("SetDKGEndState", suite.NextEpochCounter(), mock.Anything). + suite.dkgState.On("SetDKGState", suite.NextEpochCounter(), mock.Anything). Run(func(args mock.Arguments) { - assert.Equal(suite.T(), flow.DKGEndStateUnknown, suite.dkgEndState) // must be unset - endState := args[1].(flow.DKGEndState) - suite.dkgEndState = endState + assert.Equal(suite.T(), flow.DKGStateCompleted, suite.DKGState) // must be equal to the initial state of the test + endState := args[1].(flow.DKGState) + suite.DKGState = endState }). Return(nil) - suite.dkgState.On("GetDKGEndState", suite.NextEpochCounter()).Return( - func(_ uint64) flow.DKGEndState { return suite.dkgEndState }, + suite.dkgState.On("GetDKGState", suite.NextEpochCounter()).Return( + func(_ uint64) flow.DKGState { return suite.DKGState }, func(_ uint64) error { - if suite.dkgEndState == flow.DKGEndStateUnknown { + if suite.DKGState == flow.DKGStateUninitialized { return storerr.ErrNotFound } return nil @@ -382,7 +382,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestDKGSuccess() { suite.engine.EpochCommittedPhaseStarted(suite.epochCounter, suite.firstBlock) suite.Require().Equal(0, suite.warnsLogged) - suite.Assert().Equal(flow.DKGEndStateSuccess, suite.dkgEndState) + suite.Assert().Equal(flow.RandomBeaconKeyCommitted, suite.DKGState) } // TestInconsistentKey tests the path where we are checking the global DKG @@ -397,7 +397,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestInconsistentKey() { suite.engine.EpochCommittedPhaseStarted(suite.epochCounter, suite.firstBlock) suite.Require().Equal(1, suite.warnsLogged) - suite.Assert().Equal(flow.DKGEndStateInconsistentKey, suite.dkgEndState) + suite.Assert().Equal(flow.DKGStateFailure, suite.DKGState) } // TestMissingKey tests the path where we are checking the global DKG results @@ -412,7 +412,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestMissingKey() { suite.engine.EpochCommittedPhaseStarted(suite.epochCounter, suite.firstBlock) suite.Require().Equal(1, suite.warnsLogged) - suite.Assert().Equal(flow.DKGEndStateNoKey, suite.dkgEndState) + suite.Assert().Equal(flow.DKGStateFailure, suite.DKGState) } // TestLocalDKGFailure tests the path where we are checking the global DKG @@ -423,11 +423,11 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestMissingKey() { func (suite *ReactorEngineSuite_CommittedPhase) TestLocalDKGFailure() { // set dkg end state as failure - suite.dkgEndState = flow.DKGEndStateDKGFailure + suite.DKGState = flow.DKGStateFailure suite.engine.EpochCommittedPhaseStarted(suite.epochCounter, suite.firstBlock) suite.Require().Equal(1, suite.warnsLogged) - suite.Assert().Equal(flow.DKGEndStateDKGFailure, suite.dkgEndState) + suite.Assert().Equal(flow.DKGStateFailure, suite.DKGState) } // TestStartupInCommittedPhase_DKGSuccess tests that the dkg end state is correctly @@ -438,7 +438,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_DKGS suite.snap.On("EpochPhase").Return(flow.EpochPhaseCommitted, nil).Once() // the dkg for this epoch has been started but not ended suite.dkgState.On("GetDKGStarted", suite.NextEpochCounter()).Return(true, nil).Once() - suite.dkgState.On("GetDKGEndState", suite.NextEpochCounter()).Return(flow.DKGEndStateUnknown, storerr.ErrNotFound).Once() + suite.dkgState.On("GetDKGState", suite.NextEpochCounter()).Return(flow.DKGStateUninitialized, storerr.ErrNotFound).Once() // start up the engine unittest.AssertClosesBefore(suite.T(), suite.engine.Ready(), time.Second) @@ -450,18 +450,18 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_DKGS mock.Anything, ) // should set DKG end state - suite.Assert().Equal(flow.DKGEndStateSuccess, suite.dkgEndState) + suite.Assert().Equal(flow.RandomBeaconKeyCommitted, suite.DKGState) } // TestStartupInCommittedPhase_DKGSuccess tests that the dkg end state is correctly // set when starting in EpochCommitted phase and the DKG end state is already set. -func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_DKGEndStateAlreadySet() { +func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_DKGStateAlreadySet() { // we are in the EpochSetup phase suite.snap.On("EpochPhase").Return(flow.EpochPhaseCommitted, nil).Once() // the dkg for this epoch has been started and ended suite.dkgState.On("GetDKGStarted", suite.NextEpochCounter()).Return(true, nil).Once() - suite.dkgState.On("GetDKGEndState", suite.NextEpochCounter()).Return(flow.DKGEndStateNoKey, nil).Once() + suite.dkgState.On("GetDKGState", suite.NextEpochCounter()).Return(flow.DKGStateFailure, nil).Once() // start up the engine unittest.AssertClosesBefore(suite.T(), suite.engine.Ready(), time.Second) @@ -482,7 +482,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_Inco suite.snap.On("EpochPhase").Return(flow.EpochPhaseCommitted, nil).Once() // the dkg for this epoch has been started but not ended suite.dkgState.On("GetDKGStarted", suite.NextEpochCounter()).Return(true, nil).Once() - suite.dkgState.On("GetDKGEndState", suite.NextEpochCounter()).Return(flow.DKGEndStateUnknown, storerr.ErrNotFound).Once() + suite.dkgState.On("GetDKGState", suite.NextEpochCounter()).Return(flow.DKGStateUninitialized, storerr.ErrNotFound).Once() // set our global pub key to a random value suite.myGlobalBeaconPubKey = unittest.RandomBeaconPriv().PublicKey() @@ -497,7 +497,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_Inco mock.Anything, ) // should set DKG end state - suite.Assert().Equal(flow.DKGEndStateInconsistentKey, suite.dkgEndState) + suite.Assert().Equal(flow.DKGStateFailure, suite.DKGState) } // TestStartupInCommittedPhase_MissingKey tests that the dkg end state is correctly @@ -508,7 +508,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_Miss suite.snap.On("EpochPhase").Return(flow.EpochPhaseCommitted, nil).Once() // the dkg for this epoch has been started but not ended suite.dkgState.On("GetDKGStarted", suite.NextEpochCounter()).Return(true, nil).Once() - suite.dkgState.On("GetDKGEndState", suite.NextEpochCounter()).Return(flow.DKGEndStateUnknown, storerr.ErrNotFound).Once() + suite.dkgState.On("GetDKGState", suite.NextEpochCounter()).Return(flow.DKGStateUninitialized, storerr.ErrNotFound).Once() // remove our key suite.myLocalBeaconKey = nil @@ -523,7 +523,7 @@ func (suite *ReactorEngineSuite_CommittedPhase) TestStartupInCommittedPhase_Miss mock.Anything, ) // should set DKG end state - suite.Assert().Equal(flow.DKGEndStateNoKey, suite.dkgEndState) + suite.Assert().Equal(flow.DKGStateFailure, suite.DKGState) } // utility function to track the number of warn-level calls to a logger diff --git a/integration/dkg/dkg_emulator_suite.go b/integration/dkg/dkg_emulator_suite.go index 6d1677029e3..a7e99821bd7 100644 --- a/integration/dkg/dkg_emulator_suite.go +++ b/integration/dkg/dkg_emulator_suite.go @@ -449,7 +449,7 @@ func (s *EmulatorSuite) initEngines(node *node, ids flow.IdentityList) { // dkgState is used to store the private key resulting from the node's // participation in the DKG run - dkgState, err := badger.NewDKGState(core.Metrics, core.SecretsDB) + dkgState, err := badger.NewRecoverableRandomBeaconStateMachine(core.Metrics, core.SecretsDB) s.Require().NoError(err) // brokerTunnel is used to communicate between the messaging engine and the @@ -502,7 +502,6 @@ func (s *EmulatorSuite) initEngines(node *node, ids flow.IdentityList) { node.GenericNode = core node.messagingEngine = messagingEngine node.dkgState = dkgState - node.safeBeaconKeys = badger.NewSafeBeaconPrivateKeys(dkgState) node.reactorEngine = reactorEngine } diff --git a/integration/dkg/dkg_emulator_test.go b/integration/dkg/dkg_emulator_test.go index 4e61ee37127..1ffe9d8334f 100644 --- a/integration/dkg/dkg_emulator_test.go +++ b/integration/dkg/dkg_emulator_test.go @@ -136,8 +136,7 @@ func (s *EmulatorSuite) runTest(goodNodes int, emulatorProblems bool) { signatures := []crypto.Signature{} indices := []int{} for i, n := range nodes { - // TODO: to replace with safeBeaconKeys - beaconKey, err := n.dkgState.RetrieveMyBeaconPrivateKey(nextEpochSetup.Counter) + beaconKey, err := n.dkgState.UnsafeRetrieveMyBeaconPrivateKey(nextEpochSetup.Counter) require.NoError(s.T(), err) signature, err := beaconKey.Sign(sigData, hasher) diff --git a/integration/dkg/dkg_whiteboard_test.go b/integration/dkg/dkg_whiteboard_test.go index 205dd454adb..d82211771bf 100644 --- a/integration/dkg/dkg_whiteboard_test.go +++ b/integration/dkg/dkg_whiteboard_test.go @@ -91,7 +91,7 @@ func createNode( // keyKeys is used to store the private key resulting from the node's // participation in the DKG run - dkgState, err := badger.NewDKGState(core.Metrics, core.SecretsDB) + dkgState, err := badger.NewRecoverableRandomBeaconStateMachine(core.Metrics, core.SecretsDB) require.NoError(t, err) // configure the state snapthost at firstBlock to return the desired @@ -165,13 +165,10 @@ func createNode( // reactorEngine consumes the EpochSetupPhaseStarted event core.ProtocolEvents.AddConsumer(reactorEngine) - safeBeaconKeys := badger.NewSafeBeaconPrivateKeys(dkgState) - node := node{ t: t, GenericNode: core, dkgState: dkgState, - safeBeaconKeys: safeBeaconKeys, messagingEngine: messagingEngine, reactorEngine: reactorEngine, } @@ -298,9 +295,7 @@ func TestWithWhiteboard(t *testing.T) { signatures := []crypto.Signature{} indices := []int{} for i, n := range nodes { - - // TODO: to replace with safeBeaconKeys - beaconKey, err := n.dkgState.RetrieveMyBeaconPrivateKey(nextEpochSetup.Counter) + beaconKey, err := n.dkgState.UnsafeRetrieveMyBeaconPrivateKey(nextEpochSetup.Counter) require.NoError(t, err) signature, err := beaconKey.Sign(sigData, hasher) diff --git a/integration/dkg/node.go b/integration/dkg/node.go index cbea2b7f44a..2734edd40f5 100644 --- a/integration/dkg/node.go +++ b/integration/dkg/node.go @@ -4,8 +4,6 @@ import ( "crypto" "testing" - "github.com/stretchr/testify/require" - sdk "github.com/onflow/flow-go-sdk" sdkcrypto "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/engine/consensus/dkg" @@ -36,7 +34,6 @@ type node struct { account *nodeAccount dkgContractClient *DKGClientWrapper dkgState storage.DKGState - safeBeaconKeys storage.SafeBeaconKeys messagingEngine *dkg.MessagingEngine reactorEngine *dkg.ReactorEngine } @@ -54,8 +51,6 @@ func (n *node) Ready() <-chan struct{} { } func (n *node) Done() <-chan struct{} { - require.NoError(n.t, n.PublicDB.Close()) - require.NoError(n.t, n.SecretsDB.Close()) return util.AllDone(n.messagingEngine, n.reactorEngine) } diff --git a/integration/tests/access/cohort3/execution_data_pruning_test.go b/integration/tests/access/cohort3/execution_data_pruning_test.go index 312ee60347c..4c117c68da6 100644 --- a/integration/tests/access/cohort3/execution_data_pruning_test.go +++ b/integration/tests/access/cohort3/execution_data_pruning_test.go @@ -85,7 +85,7 @@ func (s *ExecutionDataPruningSuite) SetupTest() { testnet.WithAdditionalFlagf("--event-query-mode=local-only"), testnet.WithAdditionalFlagf("--execution-data-height-range-target=%d", s.heightRangeTarget), testnet.WithAdditionalFlagf("--execution-data-height-range-threshold=%d", s.threshold), - testnet.WithAdditionalFlagf(fmt.Sprintf("--execution-data-pruning-interval=%s", s.pruningInterval)), + testnet.WithAdditionalFlagf("--execution-data-pruning-interval=%s", s.pruningInterval), ) consensusConfigs := []func(config *testnet.NodeConfig){ diff --git a/model/flow/dkg.go b/model/flow/dkg.go index e9825d86fc8..05a33a5626e 100644 --- a/model/flow/dkg.go +++ b/model/flow/dkg.go @@ -1,38 +1,44 @@ package flow -// DKGEndState captures the final state of a completed DKG. -type DKGEndState uint32 +// DKGState captures all possible states of the Recoverable Random Beacon State Machine. +type DKGState uint32 const ( - // DKGEndStateUnknown - zero value for this enum, indicates unset value - DKGEndStateUnknown DKGEndState = iota - // DKGEndStateSuccess - the DKG completed, this node has a valid beacon key. - DKGEndStateSuccess - // DKGEndStateInconsistentKey - the DKG completed, this node has an invalid beacon key. - DKGEndStateInconsistentKey - // DKGEndStateNoKey - this node did not store a key, typically caused by a crash mid-DKG. - DKGEndStateNoKey - // DKGEndStateDKGFailure - the underlying DKG library reported an error. - DKGEndStateDKGFailure - // RandomBeaconKeyRecovered - this node has recovered its beacon key from a previous epoch. - // This occurs only for epochs which are entered through the EFM Recovery process (`flow.EpochRecover` service event). - RandomBeaconKeyRecovered + // DKGStateUninitialized - zero value for this enum, indicates that there is no initialized state. + // Conceptually, this is the 'initial' state of a finite state machine before any transitions. + DKGStateUninitialized DKGState = iota + // DKGStateStarted - the DKG process has been started. This state is set when the node enters the [flow.EpochPhaseSetup] + // phase and starts the DKG process, which will on the happy path result in generating a Random Beacon key. + DKGStateStarted + // DKGStateCompleted - the DKG process has been locally completed by this node. This state is set when the node successfully + // completes the DKG process and has generated a Random Beacon key. + // ATTENTION: This state does not imply that there is a safe Random Beacon key available for the next epoch. Only after + // the node enters [flow.EpochPhaseCommitted] and the [flow.EpochCommit] service event has been finalized, we can be sure + // that our beacon key share is part of the Random Beacon Committee for the next epoch, in this case the state will be [flow.RandomBeaconKeyCommitted]. + DKGStateCompleted + // RandomBeaconKeyCommitted - the Random Beacon key has been committed. This state is set when the node has observed an [flow.EpochCommit] + // which contains the public key share that matches the private key share that the node has obtained. + // A node can obtain a key share by successfully completing the DKG process or by manually injecting a key share obtained + // by other means (e.g. key recovery). + // Despite the key origin this is a terminal state which defines a safe Random Beacon key for the next epoch and allow node + // to participate in the Random Beacon protocol. + RandomBeaconKeyCommitted + // DKGStateFailure - DKG process has failed, this state indicates that we have left the happy path. + DKGStateFailure ) -func (state DKGEndState) String() string { +func (state DKGState) String() string { switch state { - case DKGEndStateSuccess: - return "DKGEndStateSuccess" - case DKGEndStateInconsistentKey: - return "DKGEndStateInconsistentKey" - case DKGEndStateNoKey: - return "DKGEndStateNoKey" - case DKGEndStateDKGFailure: - return "DKGEndStateDKGFailure" - case RandomBeaconKeyRecovered: - return "RandomBeaconKeyRecovered" + case DKGStateStarted: + return "DKGStateStarted" + case DKGStateCompleted: + return "DKGStateCompleted" + case RandomBeaconKeyCommitted: + return "RandomBeaconKeyCommitted" + case DKGStateFailure: + return "DKGStateFailure" default: - return "DKGEndStateUnknown" + return "DKGStateUninitialized" } } @@ -44,7 +50,7 @@ func (state DKGEndState) String() string { // - The values in DKGIndexMap must form the set {0, 1, …, n-1}, as required by the low level cryptography // module (convention simplifying the implementation). // -// Flow's random beacon utilizes a threshold signature scheme run by the committee 𝒟. +// Flow's Random Beacon utilizes a threshold signature scheme run by the committee 𝒟. // In the formal cryptographic protocol for a threshold signature with n parties, the // individual participants are identified by n public distinct non-negative integers, or simply indices. // These public indices are agreed upon by all participants and are used by the low-level @@ -56,7 +62,7 @@ func (state DKGEndState) String() string { // the set {0, 1, ..., n-1}. // // On the protocol level, only consensus nodes (identified by their nodeIDs) are allowed to contribute -// random beacon signature shares. Hence, the protocol level needs to map nodeIDs to the indices when +// Random Beacon signature shares. Hence, the protocol level needs to map nodeIDs to the indices when // calling into the lower-level cryptographic primitives. // // CAUTION: It is important to cleanly differentiate between the consensus committee 𝒞, the DKG committee 𝒟 @@ -68,11 +74,11 @@ func (state DKGEndState) String() string { // - The DKG committee 𝒟 is the set of parties that were authorized to participate in the DKG (happy path; or // eligible to receive a private key share from an alternative source on the fallback path). Mathematically, // the DKGIndexMap is a bijective function DKGIndexMap: 𝒟 ↦ {0,1,…,n-1}. -// - Only consensus nodes are allowed to contribute to the random beacon. Informally, we define ℛ as the +// - Only consensus nodes are allowed to contribute to the Random Beacon. Informally, we define ℛ as the // as the subset of the consensus committee (ℛ ⊆ 𝒞), which _successfully_ completed the DKG (hence ℛ ⊆ 𝒟). // Specifically, r ∈ ℛ iff and only if r has a private Random Beacon key share matching the respective public // key share in the `EpochCommit` event. In other words, consensus nodes are in ℛ iff and only if they are able -// to submit valid random beacon votes. Based on this definition we note that ℛ ⊆ (𝒟 ∩ 𝒞). +// to submit valid Random Beacon votes. Based on this definition we note that ℛ ⊆ (𝒟 ∩ 𝒞). // // The protocol explicitly ALLOWS additional parties outside the current epoch's consensus committee to participate. // In particular, there can be a key-value pair (d,i) ∈ DKGIndexMap, such that the nodeID d is *not* a consensus @@ -86,9 +92,9 @@ func (state DKGEndState) String() string { // // Nevertheless, there is an important liveness constraint: the committee ℛ should be a large number of nodes. // Specifically, an honest supermajority of consensus nodes must contain enough successful DKG participants -// (about |𝒟|/2 + 1) to produce a valid group signature for the random beacon at each block [1, 3]. +// (about |𝒟|/2 + 1) to produce a valid group signature for the Random Beacon at each block [1, 3]. // Therefore, we have the approximate lower bound |ℛ| ≳ n/2 + 1 = |𝒟|/2 + 1 = len(DKGIndexMap)/2 + 1. -// Operating close to this lower bound would require that every random beacon key-holder ϱ ∈ ℛ remaining in the consensus committee is honest +// Operating close to this lower bound would require that every Random Beacon key-holder ϱ ∈ ℛ remaining in the consensus committee is honest // (incl. quickly responsive) *all the time*. Such a reliability assumption is unsuited for decentralized production networks. // To reject configurations that are vulnerable to liveness failures, the protocol uses the threshold `t_safety` // (heuristic, see [2]), which is implemented on the smart contract level. diff --git a/module/dkg/recovery.go b/module/dkg/recovery.go index 93bc93262d5..2ad456a9fad 100644 --- a/module/dkg/recovery.go +++ b/module/dkg/recovery.go @@ -167,7 +167,7 @@ func (b *BeaconKeyRecovery) recoverMyBeaconPrivateKey(final protocol.Snapshot) e if err != nil { return fmt.Errorf("could not overwrite my beacon private key for the next epoch: %w", err) } - log.Info().Msgf("succesfully recovered my beacon private key for the next epoch") + log.Warn().Msgf("succesfully recovered my beacon private key for the next epoch") } else { log.Debug().Msgf("my beacon key is not part of the next epoch DKG") } diff --git a/storage/badger/dkg_state.go b/storage/badger/dkg_state.go index 88edc495b9f..a44d6fc23f5 100644 --- a/storage/badger/dkg_state.go +++ b/storage/badger/dkg_state.go @@ -1,10 +1,12 @@ package badger import ( + "errors" "fmt" "github.com/dgraph-io/badger/v2" "github.com/onflow/crypto" + "golang.org/x/exp/slices" "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/flow" @@ -16,15 +18,34 @@ import ( "github.com/onflow/flow-go/storage/badger/transaction" ) -// DKGState stores state information about in-progress and completed DKGs, including +// allowedStateTransitions defines the allowed state transitions for the Recoverable Random Beacon State Machine. +var allowedStateTransitions = map[flow.DKGState][]flow.DKGState{ + flow.DKGStateStarted: {flow.DKGStateCompleted, flow.DKGStateFailure, flow.RandomBeaconKeyCommitted}, + flow.DKGStateCompleted: {flow.RandomBeaconKeyCommitted, flow.DKGStateFailure, flow.RandomBeaconKeyCommitted}, + flow.RandomBeaconKeyCommitted: {flow.RandomBeaconKeyCommitted}, + flow.DKGStateFailure: {flow.RandomBeaconKeyCommitted}, + flow.DKGStateUninitialized: {flow.DKGStateStarted, flow.DKGStateFailure, flow.RandomBeaconKeyCommitted}, +} + +// RecoverablePrivateBeaconKeyStateMachine stores state information about in-progress and completed DKGs, including // computed keys. Must be instantiated using secrets database. -type DKGState struct { +// RecoverablePrivateBeaconKeyStateMachine is a specific module that allows to overwrite the beacon private key for a given epoch. +// This module is used *ONLY* in the epoch recovery process and only by the consensus participants. +// Each consensus participant takes part in the DKG, and after successfully finishing the DKG protocol it obtains a +// random beacon private key, which is stored in the database along with DKG current state [flow.DKGStateCompleted]. +// If for any reason the DKG fails, then the private key will be nil and DKG current state will be [flow.DKGStateFailure]. +// When the epoch recovery takes place, we need to query the last valid beacon private key for the current replica and +// also set it for use during the Recovery Epoch, otherwise replicas won't be able to vote for blocks during the Recovery Epoch. +type RecoverablePrivateBeaconKeyStateMachine struct { db *badger.DB keyCache *Cache[uint64, *encodable.RandomBeaconPrivKey] } -// NewDKGState returns the DKGState implementation backed by Badger DB. -func NewDKGState(collector module.CacheMetrics, db *badger.DB) (*DKGState, error) { +var _ storage.EpochRecoveryMyBeaconKey = (*RecoverablePrivateBeaconKeyStateMachine)(nil) + +// NewRecoverableRandomBeaconStateMachine returns the RecoverablePrivateBeaconKeyStateMachine implementation backed by Badger DB. +// No errors are expected during normal operations. +func NewRecoverableRandomBeaconStateMachine(collector module.CacheMetrics, db *badger.DB) (*RecoverablePrivateBeaconKeyStateMachine, error) { err := operation.EnsureSecretDB(db) if err != nil { return nil, fmt.Errorf("cannot instantiate dkg state storage in non-secret db: %w", err) @@ -48,90 +69,109 @@ func NewDKGState(collector module.CacheMetrics, db *badger.DB) (*DKGState, error withRetrieve(retrieveKey), ) - dkgState := &DKGState{ + return &RecoverablePrivateBeaconKeyStateMachine{ db: db, keyCache: cache, - } - - return dkgState, nil -} - -func (ds *DKGState) storeKeyTx(epochCounter uint64, key *encodable.RandomBeaconPrivKey) func(tx *transaction.Tx) error { - return ds.keyCache.PutTx(epochCounter, key) -} - -func (ds *DKGState) retrieveKeyTx(epochCounter uint64) func(tx *badger.Txn) (*encodable.RandomBeaconPrivKey, error) { - return func(tx *badger.Txn) (*encodable.RandomBeaconPrivKey, error) { - val, err := ds.keyCache.Get(epochCounter)(tx) - if err != nil { - return nil, err - } - return val, nil - } + }, nil } // InsertMyBeaconPrivateKey stores the random beacon private key for an epoch. // // CAUTION: these keys are stored before they are validated against the // canonical key vector and may not be valid for use in signing. Use SafeBeaconKeys -// to guarantee only keys safe for signing are returned -func (ds *DKGState) InsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error { +// to guarantee only keys safe for signing are returned. +// Error returns: +// - [storage.ErrAlreadyExists] - if there is already a key stored for given epoch. +// - [storage.InvalidTransitionRandomBeaconStateMachineErr] - if the requested state transition is invalid. +func (ds *RecoverablePrivateBeaconKeyStateMachine) InsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error { if key == nil { return fmt.Errorf("will not store nil beacon key") } encodableKey := &encodable.RandomBeaconPrivKey{PrivateKey: key} - return operation.RetryOnConflictTx(ds.db, transaction.Update, ds.storeKeyTx(epochCounter, encodableKey)) + return operation.RetryOnConflictTx(ds.db, transaction.Update, func(tx *transaction.Tx) error { + err := ds.keyCache.PutTx(epochCounter, encodableKey)(tx) + if err != nil { + return err + } + return ds.processStateTransition(epochCounter, flow.DKGStateCompleted)(tx) + }) } -// RetrieveMyBeaconPrivateKey retrieves the random beacon private key for an epoch. +// UnsafeRetrieveMyBeaconPrivateKey retrieves the random beacon private key for an epoch. // // CAUTION: these keys are stored before they are validated against the // canonical key vector and may not be valid for use in signing. Use SafeBeaconKeys // to guarantee only keys safe for signing are returned -func (ds *DKGState) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) { +// Error returns: +// - [storage.ErrNotFound] - if there is no key stored for given epoch. +func (ds *RecoverablePrivateBeaconKeyStateMachine) UnsafeRetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) { tx := ds.db.NewTransaction(false) defer tx.Discard() - encodableKey, err := ds.retrieveKeyTx(epochCounter)(tx) + encodableKey, err := ds.keyCache.Get(epochCounter)(tx) if err != nil { return nil, err } return encodableKey.PrivateKey, nil } -// SetDKGStarted sets the flag indicating the DKG has started for the given epoch. -func (ds *DKGState) SetDKGStarted(epochCounter uint64) error { - return ds.db.Update(operation.InsertDKGStartedForEpoch(epochCounter)) -} - // GetDKGStarted checks whether the DKG has been started for the given epoch. -func (ds *DKGState) GetDKGStarted(epochCounter uint64) (bool, error) { +// No errors expected during normal operation. +func (ds *RecoverablePrivateBeaconKeyStateMachine) GetDKGStarted(epochCounter uint64) (bool, error) { var started bool err := ds.db.View(operation.RetrieveDKGStartedForEpoch(epochCounter, &started)) return started, err } -// SetDKGEndState stores that the DKG has ended, and its end state. -func (ds *DKGState) SetDKGEndState(epochCounter uint64, endState flow.DKGEndState) error { - return ds.db.Update(operation.InsertDKGEndStateForEpoch(epochCounter, endState)) +// SetDKGState performs a state transition for the Random Beacon Recoverable State Machine. +// Some state transitions may not be possible using this method. For instance, we might not be able to enter [flow.DKGStateCompleted] +// state directly from [flow.DKGStateStarted], even if such transition is valid. The reason for this is that some states require additional +// data to be processed by the state machine before the transition can be made. For such cases there are dedicated methods that should be used, ex. +// InsertMyBeaconPrivateKey and UpsertMyBeaconPrivateKey, which allow to store the needed data and perform the transition in one atomic operation. +// Error returns: +// - [storage.InvalidTransitionRandomBeaconStateMachineErr] - if the requested state transition is invalid. +func (ds *RecoverablePrivateBeaconKeyStateMachine) SetDKGState(epochCounter uint64, newState flow.DKGState) error { + return operation.RetryOnConflictTx(ds.db, transaction.Update, ds.processStateTransition(epochCounter, newState)) } -// GetDKGEndState retrieves the DKG end state for the epoch. -func (ds *DKGState) GetDKGEndState(epochCounter uint64) (flow.DKGEndState, error) { - var endState flow.DKGEndState - err := ds.db.Update(operation.RetrieveDKGEndStateForEpoch(epochCounter, &endState)) - return endState, err -} +// Error returns: +// - storage.InvalidTransitionRandomBeaconStateMachineErr - if the requested state transition is invalid +func (ds *RecoverablePrivateBeaconKeyStateMachine) processStateTransition(epochCounter uint64, newState flow.DKGState) func(*transaction.Tx) error { + return func(tx *transaction.Tx) error { + var currentState flow.DKGState + err := operation.RetrieveDKGStateForEpoch(epochCounter, ¤tState)(tx.DBTxn) + if err != nil { + if errors.Is(err, storage.ErrNotFound) { + currentState = flow.DKGStateUninitialized + } else { + return fmt.Errorf("could not retrieve current state for epoch %d: %w", epochCounter, err) + } + } -// SafeBeaconPrivateKeys is the safe beacon key storage backed by Badger DB. -type SafeBeaconPrivateKeys struct { - state *DKGState -} + allowedStates := allowedStateTransitions[currentState] + if slices.Index(allowedStates, newState) < 0 { + return storage.NewInvalidTransitionRandomBeaconStateMachineErr(currentState, newState) + } + + // ensure invariant holds and we still have a valid private key stored + if newState == flow.RandomBeaconKeyCommitted || newState == flow.DKGStateCompleted { + _, err = ds.keyCache.Get(epochCounter)(tx.DBTxn) + if err != nil { + return storage.NewInvalidTransitionRandomBeaconStateMachineErrf(currentState, newState, "cannot transition without a valid random beacon key: %w", err) + } + } -var _ storage.SafeBeaconKeys = (*SafeBeaconPrivateKeys)(nil) + return operation.UpsertDKGStateForEpoch(epochCounter, newState)(tx.DBTxn) + } +} -// NewSafeBeaconPrivateKeys returns a safe beacon key storage backed by Badger DB. -func NewSafeBeaconPrivateKeys(state *DKGState) *SafeBeaconPrivateKeys { - return &SafeBeaconPrivateKeys{state: state} +// GetDKGState retrieves the current state of the state machine for the given epoch. +// If an error is returned, the state is undefined meaning that state machine is in initial state +// Error returns: +// - [storage.ErrNotFound] - if there is no state stored for given epoch, meaning the state machine is in initial state. +func (ds *RecoverablePrivateBeaconKeyStateMachine) GetDKGState(epochCounter uint64) (flow.DKGState, error) { + var currentState flow.DKGState + err := ds.db.Update(operation.RetrieveDKGStateForEpoch(epochCounter, ¤tState)) + return currentState, err } // RetrieveMyBeaconPrivateKey retrieves my beacon private key for the given @@ -141,25 +181,25 @@ func NewSafeBeaconPrivateKeys(state *DKGState) *SafeBeaconPrivateKeys { // - (key, true, nil) if the key is present and confirmed valid // - (nil, false, nil) if the key has been marked invalid or unavailable // -> no beacon key will ever be available for the epoch in this case -// - (nil, false, storage.ErrNotFound) if the DKG has not ended +// - (nil, false, [storage.ErrNotFound]) if the DKG has not ended // - (nil, false, error) for any unexpected exception -func (keys *SafeBeaconPrivateKeys) RetrieveMyBeaconPrivateKey(epochCounter uint64) (key crypto.PrivateKey, safe bool, err error) { - err = keys.state.db.View(func(txn *badger.Txn) error { +func (ds *RecoverablePrivateBeaconKeyStateMachine) RetrieveMyBeaconPrivateKey(epochCounter uint64) (key crypto.PrivateKey, safe bool, err error) { + err = ds.db.View(func(txn *badger.Txn) error { // retrieve the end state - var endState flow.DKGEndState - err = operation.RetrieveDKGEndStateForEpoch(epochCounter, &endState)(txn) + var currentState flow.DKGState + err = operation.RetrieveDKGStateForEpoch(epochCounter, ¤tState)(txn) if err != nil { key = nil safe = false return err // storage.ErrNotFound or exception } - // for any end state besides success and recovery, the key is not safe - if endState == flow.DKGEndStateSuccess || endState == flow.RandomBeaconKeyRecovered { + // a key is safe iff it was previously committed + if currentState == flow.RandomBeaconKeyCommitted { // retrieve the key - any storage error (including `storage.ErrNotFound`) is an exception var encodableKey *encodable.RandomBeaconPrivKey - encodableKey, err = keys.state.retrieveKeyTx(epochCounter)(txn) + encodableKey, err = ds.keyCache.Get(epochCounter)(txn) if err != nil { key = nil safe = false @@ -172,6 +212,7 @@ func (keys *SafeBeaconPrivateKeys) RetrieveMyBeaconPrivateKey(epochCounter uint6 } else { key = nil safe = false + return storage.ErrNotFound } return nil @@ -179,42 +220,25 @@ func (keys *SafeBeaconPrivateKeys) RetrieveMyBeaconPrivateKey(epochCounter uint6 return } -// EpochRecoveryMyBeaconKey is a specific module that allows to overwrite the beacon private key for a given epoch. -// This module is used *ONLY* in the epoch recovery process and only by the consensus participants. -// Each consensus participant takes part in the DKG, and after successfully finishing the DKG protocol it obtains a -// random beacon private key, which is stored in the database along with DKG end state `flow.DKGEndStateSuccess`. -// If for any reason the DKG fails, then the private key will be nil and DKG end state will be `flow.DKGEndStateDKGFailure`. -// When the epoch recovery takes place, we need to query the last valid beacon private key for the current replica and -// also set it for use during the Recovery Epoch, otherwise replicas won't be able to vote for blocks during the Recovery Epoch. -type EpochRecoveryMyBeaconKey struct { - *SafeBeaconPrivateKeys -} - -var _ storage.EpochRecoveryMyBeaconKey = (*EpochRecoveryMyBeaconKey)(nil) - -func NewEpochRecoveryMyBeaconKey(keys *SafeBeaconPrivateKeys) *EpochRecoveryMyBeaconKey { - return &EpochRecoveryMyBeaconKey{SafeBeaconPrivateKeys: keys} -} - // UpsertMyBeaconPrivateKey overwrites the random beacon private key for the epoch that recovers the protocol from // Epoch Fallback Mode. Effectively, this function overwrites whatever might be available in the database with -// the given private key and sets the DKGEndState to `flow.DKGEndStateRecovered`. +// the given private key and sets the [flow.DKGState] to [flow.RandomBeaconKeyCommitted]. // No errors are expected during normal operations. -func (keys *EpochRecoveryMyBeaconKey) UpsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error { +func (ds *RecoverablePrivateBeaconKeyStateMachine) UpsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error { if key == nil { return fmt.Errorf("will not store nil beacon key") } encodableKey := &encodable.RandomBeaconPrivKey{PrivateKey: key} - err := keys.state.db.Update(func(txn *badger.Txn) error { - err := operation.UpsertMyBeaconPrivateKey(epochCounter, encodableKey)(txn) + err := operation.RetryOnConflictTx(ds.db, transaction.Update, func(tx *transaction.Tx) error { + err := operation.UpsertMyBeaconPrivateKey(epochCounter, encodableKey)(tx.DBTxn) if err != nil { return err } - return operation.UpsertDKGEndStateForEpoch(epochCounter, flow.RandomBeaconKeyRecovered)(txn) + return ds.processStateTransition(epochCounter, flow.RandomBeaconKeyCommitted)(tx) }) if err != nil { return fmt.Errorf("could not overwrite beacon key for epoch %d: %w", epochCounter, err) } - keys.state.keyCache.Insert(epochCounter, encodableKey) + ds.keyCache.Insert(epochCounter, encodableKey) return nil } diff --git a/storage/badger/dkg_state_test.go b/storage/badger/dkg_state_test.go index 5643b064d22..f294a2090a3 100644 --- a/storage/badger/dkg_state_test.go +++ b/storage/badger/dkg_state_test.go @@ -1,232 +1,391 @@ -package badger_test +package badger import ( - "errors" "math/rand" "testing" "github.com/dgraph-io/badger/v2" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/storage" - bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/utils/unittest" ) -func TestDKGState_DKGStarted(t *testing.T) { - unittest.RunWithTypedBadgerDB(t, bstorage.InitSecret, func(db *badger.DB) { +// TestDKGState_UninitializedState checks that invariants are enforced for [flow.DKGStateUninitialized] state. +// This test is written in a way that we start with initial state of the Recoverable Random Beacon State Machine and +// try to perform all possible actions and transitions in it. +func TestDKGState_UninitializedState(t *testing.T) { + unittest.RunWithTypedBadgerDB(t, InitSecret, func(db *badger.DB) { metrics := metrics.NewNoopCollector() - store, err := bstorage.NewDKGState(metrics, db) + store, err := NewRecoverableRandomBeaconStateMachine(metrics, db) require.NoError(t, err) - epochCounter := rand.Uint64() + setupState := func() uint64 { + return rand.Uint64() + } + epochCounter := setupState() - // check dkg-started flag for non-existent epoch - t.Run("DKGStarted should default to false", func(t *testing.T) { - started, err := store.GetDKGStarted(rand.Uint64()) - assert.NoError(t, err) - assert.False(t, started) + started, err := store.GetDKGStarted(epochCounter) + require.NoError(t, err) + require.False(t, started) + + actualState, err := store.GetDKGState(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.Equal(t, flow.DKGStateUninitialized, actualState) + + pk, err := store.UnsafeRetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.Nil(t, pk) + + pk, safe, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.False(t, safe) + require.Nil(t, pk) + + t.Run("-> flow.DKGStateUninitialized, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateUninitialized) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) }) - // store dkg-started flag for epoch - t.Run("should be able to set DKGStarted", func(t *testing.T) { - err = store.SetDKGStarted(epochCounter) - assert.NoError(t, err) + t.Run("-> flow.DKGStateStarted, should be allowed", func(t *testing.T) { + epochCounter++ + err = store.SetDKGState(setupState(), flow.DKGStateStarted) + require.NoError(t, err) + }) + + t.Run("-> flow.DKGStateFailure, should be allowed", func(t *testing.T) { + epochCounter++ + err = store.SetDKGState(setupState(), flow.DKGStateFailure) + require.NoError(t, err) }) - // retrieve flag for epoch - t.Run("should be able to read DKGStarted", func(t *testing.T) { - started, err := store.GetDKGStarted(epochCounter) - assert.NoError(t, err) - assert.True(t, started) + t.Run("-> flow.DKGStateCompleted, not allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.InsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.Error(t, err, "should not be able to enter completed state without starting") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.SetDKGState(epochCounter, flow.DKGStateCompleted) + require.Error(t, err, "should not be able to enter completed state without starting") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) + + t.Run("-> flow.RandomBeaconKeyCommitted, should be allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.RandomBeaconKeyCommitted) + require.Error(t, err, "should not be able to set DKG state to recovered, only using dedicated interface") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.UpsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.NoError(t, err) }) }) } -func TestDKGState_BeaconKeys(t *testing.T) { - unittest.RunWithTypedBadgerDB(t, bstorage.InitSecret, func(db *badger.DB) { +// TestDKGState_StartedState checks that invariants are enforced for [flow.DKGStateStarted] state. +// This test is written in a way that we start in [flow.DKGStateStarted] of the Recoverable Random Beacon State Machine and +// try to perform all possible actions and transitions in it. +func TestDKGState_StartedState(t *testing.T) { + unittest.RunWithTypedBadgerDB(t, InitSecret, func(db *badger.DB) { metrics := metrics.NewNoopCollector() - store, err := bstorage.NewDKGState(metrics, db) + store, err := NewRecoverableRandomBeaconStateMachine(metrics, db) + require.NoError(t, err) + + setupState := func() uint64 { + epochCounter := rand.Uint64() + err = store.SetDKGState(epochCounter, flow.DKGStateStarted) + require.NoError(t, err) + return epochCounter + } + epochCounter := setupState() + + actualState, err := store.GetDKGState(epochCounter) + require.NoError(t, err, storage.ErrNotFound) + require.Equal(t, flow.DKGStateStarted, actualState) + + started, err := store.GetDKGStarted(epochCounter) require.NoError(t, err) + require.True(t, started) + + pk, err := store.UnsafeRetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.Nil(t, pk) - epochCounter := rand.Uint64() + pk, safe, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.False(t, safe) + require.Nil(t, pk) - // attempt to get a non-existent key - t.Run("should error if retrieving non-existent key", func(t *testing.T) { - _, err = store.RetrieveMyBeaconPrivateKey(epochCounter) - assert.True(t, errors.Is(err, storage.ErrNotFound)) + t.Run("-> flow.DKGStateUninitialized, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateUninitialized) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) }) - // attempt to store a nil key should fail - use DKGState.SetEndState(flow.DKGEndStateNoKey) - t.Run("should fail to store a nil key instead)", func(t *testing.T) { - err = store.InsertMyBeaconPrivateKey(epochCounter, nil) - assert.Error(t, err) + t.Run("-> flow.DKGStateStarted, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateStarted) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) }) - // store a key in db - expected := unittest.RandomBeaconPriv() - t.Run("should be able to store and read a key", func(t *testing.T) { - err = store.InsertMyBeaconPrivateKey(epochCounter, expected) + t.Run("-> flow.DKGStateFailure, should be allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateFailure) require.NoError(t, err) }) - // retrieve the key by epoch counter - t.Run("should be able to retrieve stored key", func(t *testing.T) { - actual, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + t.Run("-> flow.DKGStateCompleted, should be allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.DKGStateCompleted) + require.Error(t, err, "should not be able to enter completed state without providing a private key") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.InsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) require.NoError(t, err) - assert.Equal(t, expected, actual) }) - // test storing same key - t.Run("should fail to store a key twice", func(t *testing.T) { - err = store.InsertMyBeaconPrivateKey(epochCounter, expected) - require.True(t, errors.Is(err, storage.ErrAlreadyExists)) + t.Run("-> flow.RandomBeaconKeyCommitted, should be allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.RandomBeaconKeyCommitted) + require.Error(t, err, "should not be able to set DKG state to recovered, only using dedicated interface") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.UpsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.NoError(t, err) }) }) } -func TestDKGState_EndState(t *testing.T) { - unittest.RunWithTypedBadgerDB(t, bstorage.InitSecret, func(db *badger.DB) { +// TestDKGState_CompletedState checks that invariants are enforced for [flow.DKGStateCompleted] state. +// This test is written in a way that we start in [flow.DKGStateCompleted] of the Recoverable Random Beacon State Machine and +// try to perform all possible actions and transitions in it. We enter [flow.DKGStateCompleted] by inserting a mock private key. +func TestDKGState_CompletedState(t *testing.T) { + unittest.RunWithTypedBadgerDB(t, InitSecret, func(db *badger.DB) { metrics := metrics.NewNoopCollector() - store, err := bstorage.NewDKGState(metrics, db) + store, err := NewRecoverableRandomBeaconStateMachine(metrics, db) require.NoError(t, err) - epochCounter := rand.Uint64() - endState := flow.DKGEndStateNoKey + setupState := func() uint64 { + epochCounter := rand.Uint64() + err = store.SetDKGState(epochCounter, flow.DKGStateStarted) + require.NoError(t, err) + err = store.InsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.NoError(t, err) + return epochCounter + } + epochCounter := setupState() + + actualState, err := store.GetDKGState(epochCounter) + require.NoError(t, err, storage.ErrNotFound) + require.Equal(t, flow.DKGStateCompleted, actualState) + + started, err := store.GetDKGStarted(epochCounter) + require.NoError(t, err) + require.True(t, started) - t.Run("should be able to store an end state", func(t *testing.T) { - err = store.SetDKGEndState(epochCounter, endState) + pk, err := store.UnsafeRetrieveMyBeaconPrivateKey(epochCounter) + require.NoError(t, err) + require.NotNil(t, pk) + + pk, safe, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.False(t, safe) + require.Nil(t, pk) + + t.Run("-> flow.DKGStateUninitialized, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateUninitialized) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) + + t.Run("-> flow.DKGStateStarted, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateStarted) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) + + t.Run("-> flow.DKGStateFailure, should be allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateFailure) require.NoError(t, err) }) - t.Run("should be able to read an end state", func(t *testing.T) { - readEndState, err := store.GetDKGEndState(epochCounter) + t.Run("-> flow.DKGStateCompleted, not allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.DKGStateCompleted) + require.Error(t, err, "already in this state") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.InsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.Error(t, err, "already inserted private key") + require.ErrorIs(t, err, storage.ErrAlreadyExists) + }) + + t.Run("-> flow.RandomBeaconKeyCommitted, should be allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.RandomBeaconKeyCommitted) + require.NoError(t, err, "should be allowed since we have a stored private key") + }) + + t.Run("-> flow.RandomBeaconKeyCommitted(recovery), should be allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.UpsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) require.NoError(t, err) - assert.Equal(t, endState, readEndState) }) }) } -func TestSafeBeaconPrivateKeys(t *testing.T) { - unittest.RunWithTypedBadgerDB(t, bstorage.InitSecret, func(db *badger.DB) { +// TestDKGState_FailureState checks that invariants are enforced for [flow.DKGStateFailure] state. +// This test is written in a way that we start with [flow.DKGStateFailure] of the Recoverable Random Beacon State Machine and +// try to perform all possible actions and transitions in it. +func TestDKGState_FailureState(t *testing.T) { + unittest.RunWithTypedBadgerDB(t, InitSecret, func(db *badger.DB) { metrics := metrics.NewNoopCollector() - dkgState, err := bstorage.NewDKGState(metrics, db) + store, err := NewRecoverableRandomBeaconStateMachine(metrics, db) require.NoError(t, err) - safeKeys := bstorage.NewSafeBeaconPrivateKeys(dkgState) - t.Run("non-existent key -> should return ErrNotFound", func(t *testing.T) { + setupState := func() uint64 { epochCounter := rand.Uint64() - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.Nil(t, key) - assert.False(t, safe) - assert.ErrorIs(t, err, storage.ErrNotFound) - }) + err = store.SetDKGState(epochCounter, flow.DKGStateFailure) + require.NoError(t, err) + return epochCounter + } + epochCounter := setupState() - t.Run("existent key, non-existent end state -> should return ErrNotFound", func(t *testing.T) { - epochCounter := rand.Uint64() + actualState, err := store.GetDKGState(epochCounter) + require.NoError(t, err) + require.Equal(t, flow.DKGStateFailure, actualState) - // store a key - expected := unittest.RandomBeaconPriv().PrivateKey - err := dkgState.InsertMyBeaconPrivateKey(epochCounter, expected) - assert.NoError(t, err) + started, err := store.GetDKGStarted(epochCounter) + require.NoError(t, err) + require.True(t, started) - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.Nil(t, key) - assert.False(t, safe) - assert.ErrorIs(t, err, storage.ErrNotFound) - }) + pk, err := store.UnsafeRetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.Nil(t, pk) - t.Run("existent key, unsuccessful end state -> not safe", func(t *testing.T) { - epochCounter := rand.Uint64() + pk, safe, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + require.ErrorIs(t, err, storage.ErrNotFound) + require.False(t, safe) + require.Nil(t, pk) - // store a key - expected := unittest.RandomBeaconPriv().PrivateKey - err := dkgState.InsertMyBeaconPrivateKey(epochCounter, expected) - assert.NoError(t, err) - // mark dkg unsuccessful - err = dkgState.SetDKGEndState(epochCounter, flow.DKGEndStateInconsistentKey) - assert.NoError(t, err) + t.Run("-> flow.DKGStateUninitialized, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateUninitialized) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.Nil(t, key) - assert.False(t, safe) - assert.NoError(t, err) + t.Run("-> flow.DKGStateStarted, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateStarted) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) }) - t.Run("existent key, inconsistent key end state -> not safe", func(t *testing.T) { - epochCounter := rand.Uint64() + t.Run("-> flow.DKGStateFailure, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateFailure) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) - // store a key - expected := unittest.RandomBeaconPriv().PrivateKey - err := dkgState.InsertMyBeaconPrivateKey(epochCounter, expected) - assert.NoError(t, err) - // mark dkg result as inconsistent - err = dkgState.SetDKGEndState(epochCounter, flow.DKGEndStateInconsistentKey) - assert.NoError(t, err) + t.Run("-> flow.DKGStateCompleted, not allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.DKGStateCompleted) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.InsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.Nil(t, key) - assert.False(t, safe) - assert.NoError(t, err) + t.Run("-> flow.RandomBeaconKeyCommitted, should be allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.RandomBeaconKeyCommitted) + require.Error(t, err, "should not be able to set DKG state to recovered, only using dedicated interface") + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + expectedKey := unittest.RandomBeaconPriv() + err = store.UpsertMyBeaconPrivateKey(epochCounter, expectedKey) + require.NoError(t, err) + actualKey, safe, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + require.NoError(t, err) + require.True(t, safe) + require.Equal(t, expectedKey, actualKey) }) + }) +} + +// TestDKGState_RandomBeaconKeyCommittedState checks that invariants are enforced for [flow.RandomBeaconKeyCommitted] state. +// This test is written in a way that we start with [flow.RandomBeaconKeyCommitted] state of the Recoverable Random Beacon State Machine and +// try to perform all possible actions and transitions in it. +func TestDKGState_RandomBeaconKeyCommittedState(t *testing.T) { + unittest.RunWithTypedBadgerDB(t, InitSecret, func(db *badger.DB) { + metrics := metrics.NewNoopCollector() + store, err := NewRecoverableRandomBeaconStateMachine(metrics, db) + require.NoError(t, err) - t.Run("non-existent key, no key end state -> not safe", func(t *testing.T) { + setupState := func() uint64 { epochCounter := rand.Uint64() + err = store.UpsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.NoError(t, err) + return epochCounter + } + epochCounter := setupState() - // mark dkg result as no key - err = dkgState.SetDKGEndState(epochCounter, flow.DKGEndStateNoKey) - assert.NoError(t, err) + actualState, err := store.GetDKGState(epochCounter) + require.NoError(t, err) + require.Equal(t, flow.RandomBeaconKeyCommitted, actualState) - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.Nil(t, key) - assert.False(t, safe) - assert.NoError(t, err) - }) + started, err := store.GetDKGStarted(epochCounter) + require.NoError(t, err) + require.True(t, started) - t.Run("existent key, successful end state -> safe", func(t *testing.T) { - epochCounter := rand.Uint64() + pk, err := store.UnsafeRetrieveMyBeaconPrivateKey(epochCounter) + require.NoError(t, err) + require.NotNil(t, pk) - // store a key - expected := unittest.RandomBeaconPriv().PrivateKey - err := dkgState.InsertMyBeaconPrivateKey(epochCounter, expected) - assert.NoError(t, err) - // mark dkg successful - err = dkgState.SetDKGEndState(epochCounter, flow.DKGEndStateSuccess) - assert.NoError(t, err) + pk, safe, err := store.RetrieveMyBeaconPrivateKey(epochCounter) + require.NoError(t, err) + require.True(t, safe) + require.NotNil(t, pk) - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.NotNil(t, key) - assert.True(t, expected.Equals(key)) - assert.True(t, safe) - assert.NoError(t, err) + t.Run("-> flow.DKGStateUninitialized, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateUninitialized) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) }) - t.Run("non-existent key, successful end state -> exception!", func(t *testing.T) { - epochCounter := rand.Uint64() + t.Run("-> flow.DKGStateStarted, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateStarted) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) - // mark dkg successful - err = dkgState.SetDKGEndState(epochCounter, flow.DKGEndStateSuccess) - assert.NoError(t, err) + t.Run("-> flow.DKGStateFailure, not allowed", func(t *testing.T) { + err = store.SetDKGState(setupState(), flow.DKGStateFailure) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + }) - key, safe, err := safeKeys.RetrieveMyBeaconPrivateKey(epochCounter) - assert.Nil(t, key) - assert.False(t, safe) - assert.Error(t, err) - assert.NotErrorIs(t, err, storage.ErrNotFound) + t.Run("-> flow.DKGStateCompleted, not allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.DKGStateCompleted) + require.Error(t, err) + require.True(t, storage.IsInvalidTransitionRandomBeaconStateMachineErr(err)) + err = store.InsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.ErrorIs(t, err, storage.ErrAlreadyExists) }) + t.Run("-> flow.RandomBeaconKeyCommitted, allowed", func(t *testing.T) { + epochCounter := setupState() + err = store.SetDKGState(epochCounter, flow.RandomBeaconKeyCommitted) + require.NoError(t, err, "should be possible since we have a stored private key") + err = store.UpsertMyBeaconPrivateKey(epochCounter, unittest.RandomBeaconPriv()) + require.NoError(t, err) + }) }) } -// TestSecretDBRequirement tests that the DKGState constructor will return an +// TestSecretDBRequirement tests that the RecoverablePrivateBeaconKeyStateMachine constructor will return an // error if instantiated using a database not marked with the correct type. func TestSecretDBRequirement(t *testing.T) { unittest.RunWithBadgerDB(t, func(db *badger.DB) { metrics := metrics.NewNoopCollector() - _, err := bstorage.NewDKGState(metrics, db) + _, err := NewRecoverableRandomBeaconStateMachine(metrics, db) require.Error(t, err) }) } diff --git a/storage/badger/operation/dkg.go b/storage/badger/operation/dkg.go index 10a35355545..8a6dbcd65bf 100644 --- a/storage/badger/operation/dkg.go +++ b/storage/badger/operation/dkg.go @@ -41,19 +41,13 @@ func RetrieveMyBeaconPrivateKey(epochCounter uint64, info *encodable.RandomBeaco return retrieve(makePrefix(codeBeaconPrivateKey, epochCounter), info) } -// InsertDKGStartedForEpoch stores a flag indicating that the DKG has been started for the given epoch. -// Returns: storage.ErrAlreadyExists -// Error returns: storage.ErrAlreadyExists -func InsertDKGStartedForEpoch(epochCounter uint64) func(*badger.Txn) error { - return insert(makePrefix(codeDKGStarted, epochCounter), true) -} - -// RetrieveDKGStartedForEpoch retrieves the DKG started flag for the given epoch. +// RetrieveDKGStartedForEpoch retrieves whether DKG has started for the given epoch. // If no flag is set, started is set to false and no error is returned. // No errors expected during normal operation. func RetrieveDKGStartedForEpoch(epochCounter uint64, started *bool) func(*badger.Txn) error { return func(tx *badger.Txn) error { - err := retrieve(makePrefix(codeDKGStarted, epochCounter), started)(tx) + var state flow.DKGState + err := RetrieveDKGStateForEpoch(epochCounter, &state)(tx) if errors.Is(err, storage.ErrNotFound) { // flag not set - therefore DKG not started *started = false @@ -62,27 +56,29 @@ func RetrieveDKGStartedForEpoch(epochCounter uint64, started *bool) func(*badger // storage error - set started to zero value *started = false return err + } else { + *started = true } return nil } } -// InsertDKGEndStateForEpoch stores the DKG end state for the epoch. -// Error returns: storage.ErrAlreadyExists -func InsertDKGEndStateForEpoch(epochCounter uint64, endState flow.DKGEndState) func(*badger.Txn) error { - return insert(makePrefix(codeDKGEnded, epochCounter), endState) +// InsertDKGStateForEpoch stores the DKG current state of Random Beacon Recoverable State Machine for the epoch. +// Error returns: [storage.ErrAlreadyExists] +func InsertDKGStateForEpoch(epochCounter uint64, newState flow.DKGState) func(*badger.Txn) error { + return insert(makePrefix(codeDKGState, epochCounter), newState) } -// UpsertDKGEndStateForEpoch stores the DKG end state for the epoch, irrespective of whether an entry for +// UpsertDKGStateForEpoch stores the current state of Random Beacon Recoverable State Machine for the epoch, irrespective of whether an entry for // the given epoch counter already exists in the database or not. // CAUTION: this method has to be used only in the very specific edge-cases of epoch recovery. For storing the -// DKG results obtained on the happy-path, please use method `InsertDKGEndStateForEpoch`. -func UpsertDKGEndStateForEpoch(epochCounter uint64, endState flow.DKGEndState) func(*badger.Txn) error { - return upsert(makePrefix(codeDKGEnded, epochCounter), endState) +// DKG results obtained on the happy-path, please use method [InsertDKGStateForEpoch]. +func UpsertDKGStateForEpoch(epochCounter uint64, newState flow.DKGState) func(*badger.Txn) error { + return upsert(makePrefix(codeDKGState, epochCounter), newState) } -// RetrieveDKGEndStateForEpoch retrieves the DKG end state for the epoch. -// Error returns: storage.ErrNotFound -func RetrieveDKGEndStateForEpoch(epochCounter uint64, endState *flow.DKGEndState) func(*badger.Txn) error { - return retrieve(makePrefix(codeDKGEnded, epochCounter), endState) +// RetrieveDKGStateForEpoch retrieves the DKG end state for the epoch. +// Error returns: [storage.ErrNotFound] +func RetrieveDKGStateForEpoch(epochCounter uint64, newState *flow.DKGState) func(*badger.Txn) error { + return retrieve(makePrefix(codeDKGState, epochCounter), newState) } diff --git a/storage/badger/operation/dkg_test.go b/storage/badger/operation/dkg_test.go index 03417e963f6..2cf50b74bb2 100644 --- a/storage/badger/operation/dkg_test.go +++ b/storage/badger/operation/dkg_test.go @@ -61,7 +61,7 @@ func TestDKGStartedForEpoch(t *testing.T) { epochCounter := rand.Uint64() // set the flag, ensure no error - err := db.Update(InsertDKGStartedForEpoch(epochCounter)) + err := db.Update(InsertDKGStateForEpoch(epochCounter, flow.DKGStateStarted)) assert.NoError(t, err) // read the flag, should be true now @@ -83,18 +83,18 @@ func TestDKGEndStateForEpoch(t *testing.T) { epochCounter := rand.Uint64() // should be able to write end state - endState := flow.DKGEndStateSuccess - err := db.Update(InsertDKGEndStateForEpoch(epochCounter, endState)) + endState := flow.DKGStateStarted + err := db.Update(InsertDKGStateForEpoch(epochCounter, endState)) assert.NoError(t, err) // should be able to read end state - var readEndState flow.DKGEndState - err = db.View(RetrieveDKGEndStateForEpoch(epochCounter, &readEndState)) + var readEndState flow.DKGState + err = db.View(RetrieveDKGStateForEpoch(epochCounter, &readEndState)) assert.NoError(t, err) assert.Equal(t, endState, readEndState) // attempting to overwrite should error - err = db.Update(InsertDKGEndStateForEpoch(epochCounter, flow.DKGEndStateDKGFailure)) + err = db.Update(InsertDKGStateForEpoch(epochCounter, flow.DKGStateFailure)) assert.ErrorIs(t, err, storage.ErrAlreadyExists) }) } diff --git a/storage/badger/operation/prefix.go b/storage/badger/operation/prefix.go index 6170cad34ec..ea74552933a 100644 --- a/storage/badger/operation/prefix.go +++ b/storage/badger/operation/prefix.go @@ -72,8 +72,8 @@ const ( codeEpochSetup = 61 // EpochSetup service event, keyed by ID codeEpochCommit = 62 // EpochCommit service event, keyed by ID codeBeaconPrivateKey = 63 // BeaconPrivateKey, keyed by epoch counter - codeDKGStarted = 64 // flag that the DKG for an epoch has been started - codeDKGEnded = 65 // flag that the DKG for an epoch has ended (stores end state) + _ = 64 // [DEPRECATED] flag that the DKG for an epoch has been started + codeDKGState = 65 // current state of Recoverable Random Beacon State Machine for given epoch codeVersionBeacon = 67 // flag for storing version beacons codeEpochProtocolState = 68 codeProtocolKVStore = 69 diff --git a/storage/dkg.go b/storage/dkg.go index 4a13c1b92d3..db7fe977e9e 100644 --- a/storage/dkg.go +++ b/storage/dkg.go @@ -1,77 +1,125 @@ package storage import ( + "errors" + "fmt" + "github.com/onflow/crypto" "github.com/onflow/flow-go/model/flow" ) -// DKGState is the storage interface for storing all artifacts and state -// related to the DKG process, including the latest state of a running or -// completed DKG, and computed beacon keys. -type DKGState interface { +// SafeBeaconKeys is a safe way to access beacon keys. +type SafeBeaconKeys interface { - // SetDKGStarted sets the flag indicating the DKG has started for the given epoch. - // Error returns: storage.ErrAlreadyExists - SetDKGStarted(epochCounter uint64) error + // RetrieveMyBeaconPrivateKey retrieves my beacon private key for the given + // epoch, only if my key has been confirmed valid and safe for use. + // + // Returns: + // - (key, true, nil) if the key is present and confirmed valid + // - (nil, false, nil) if the key has been marked invalid or unavailable + // -> no beacon key will ever be available for the epoch in this case + // - (nil, false, [storage.ErrNotFound]) if the DKG has not ended + // - (nil, false, error) for any unexpected exception + RetrieveMyBeaconPrivateKey(epochCounter uint64) (key crypto.PrivateKey, safe bool, err error) +} + +// DKGStateReader is a ready-only interface for reading state of the Random Beacon Recoverable State Machine. +type DKGStateReader interface { + SafeBeaconKeys + + // GetDKGState retrieves the current state of the state machine for the given epoch. + // If an error is returned, the state is undefined meaning that state machine is in initial state + // Error returns: + // - [storage.ErrNotFound] - if there is no state stored for given epoch, meaning the state machine is in initial state. + GetDKGState(epochCounter uint64) (flow.DKGState, error) // GetDKGStarted checks whether the DKG has been started for the given epoch. // No errors expected during normal operation. GetDKGStarted(epochCounter uint64) (bool, error) - // SetDKGEndState stores that the DKG has ended, and its end state. - // Error returns: storage.ErrAlreadyExists - SetDKGEndState(epochCounter uint64, endState flow.DKGEndState) error - - // GetDKGEndState retrieves the end state for the given DKG. - // Error returns: storage.ErrNotFound - GetDKGEndState(epochCounter uint64) (flow.DKGEndState, error) - - // InsertMyBeaconPrivateKey stores the random beacon private key for an epoch. + // UnsafeRetrieveMyBeaconPrivateKey retrieves the random beacon private key for an epoch. // // CAUTION: these keys are stored before they are validated against the // canonical key vector and may not be valid for use in signing. Use SafeBeaconKeys // to guarantee only keys safe for signing are returned - // Error returns: storage.ErrAlreadyExists - InsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error + // Error returns: + // - [storage.ErrNotFound] - if there is no key stored for given epoch. + UnsafeRetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) +} + +// DKGState is the storage interface for storing all artifacts and state +// related to the DKG process, including the latest state of a running or completed DKG, and computed beacon keys. +// It allows to initiate state transitions to the Random Beacon Recoverable State Machine by calling respective methods. +// It supports all state transitions for the happy path. Recovery from the epoch fallback mode is supported by the EpochRecoveryMyBeaconKey interface. +type DKGState interface { + DKGStateReader + + // SetDKGState performs a state transition for the Random Beacon Recoverable State Machine. + // Some state transitions may not be possible using this method. For instance, we might not be able to enter [flow.DKGStateCompleted] + // state directly from [flow.DKGStateStarted], even if such transition is valid. The reason for this is that some states require additional + // data to be processed by the state machine before the transition can be made. For such cases there are dedicated methods that should be used, ex. + // InsertMyBeaconPrivateKey and UpsertMyBeaconPrivateKey, which allow to store the needed data and perform the transition in one atomic operation. + // Error returns: + // - [storage.InvalidTransitionRandomBeaconStateMachineErr] - if the requested state transition is invalid. + SetDKGState(epochCounter uint64, newState flow.DKGState) error - // RetrieveMyBeaconPrivateKey retrieves the random beacon private key for an epoch. + // InsertMyBeaconPrivateKey stores the random beacon private key for an epoch. // // CAUTION: these keys are stored before they are validated against the // canonical key vector and may not be valid for use in signing. Use SafeBeaconKeys // to guarantee only keys safe for signing are returned - // Error returns: storage.ErrNotFound - RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) -} - -// SafeBeaconKeys is a safe way to access beacon keys. -type SafeBeaconKeys interface { - - // RetrieveMyBeaconPrivateKey retrieves my beacon private key for the given - // epoch, only if my key has been confirmed valid and safe for use. - // - // Returns: - // - (key, true, nil) if the key is present and confirmed valid - // - (nil, false, nil) if the key has been marked invalid or unavailable - // -> no beacon key will ever be available for the epoch in this case - // - (nil, false, storage.ErrNotFound) if the DKG has not ended - // - (nil, false, error) for any unexpected exception - RetrieveMyBeaconPrivateKey(epochCounter uint64) (key crypto.PrivateKey, safe bool, err error) + // Error returns: + // - [storage.ErrAlreadyExists] - if there is already a key stored for given epoch. + // - [storage.InvalidTransitionRandomBeaconStateMachineErr] - if the requested state transition is invalid. + InsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error } -// EpochRecoveryMyBeaconKey is a specific interface that allows to overwrite the beacon private key for a given epoch. +// EpochRecoveryMyBeaconKey is a specific interface that allows to overwrite the beacon private key for given epoch. // This interface is used *ONLY* in the epoch recovery process and only by the consensus participants. -// Each consensus participant takes part in the DKG, and after successfully finishing the DKG protocol it obtains a -// random beacon private key, which is stored in the database along with DKG end state `flow.DKGEndStateSuccess`. -// If for any reason the DKG fails, then the private key will be nil and DKG end state will be `flow.DKGEndStateDKGFailure`. -// When the epoch recovery takes place, we need to query the last valid beacon private key for the current replica and -// also set it for use during the Recovery Epoch, otherwise replicas won't be able to vote for blocks during the Recovery Epoch. +// Each consensus participant takes part in the DKG, after finishing the DKG protocol each replica obtains a random beacon +// private key which is stored in the database along with DKG state which will be equal to [flow.RandomBeaconKeyCommitted]. +// If for any reason DKG fails, then the private key will be nil and DKG end state will be equal to [flow.DKGStateFailure]. +// This module allows to overwrite the random beacon private key in case of EFM recovery or other configuration issues. type EpochRecoveryMyBeaconKey interface { - SafeBeaconKeys + DKGStateReader // UpsertMyBeaconPrivateKey overwrites the random beacon private key for the epoch that recovers the protocol from // Epoch Fallback Mode. Effectively, this function overwrites whatever might be available in the database with - // the given private key and sets the DKGEndState to `flow.DKGEndStateRecovered`. + // the given private key and sets the [flow.DKGState] to [flow.RandomBeaconKeyCommitted]. // No errors are expected during normal operations. UpsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error } + +// InvalidTransitionRandomBeaconStateMachineErr is a sentinel error that is returned in case an invalid state transition is attempted. +type InvalidTransitionRandomBeaconStateMachineErr struct { + err error + From flow.DKGState + To flow.DKGState +} + +func (e InvalidTransitionRandomBeaconStateMachineErr) Error() string { + return fmt.Sprintf("invalid state transition from %s to %s: %s", e.From.String(), e.To.String(), e.err.Error()) +} + +func IsInvalidTransitionRandomBeaconStateMachineErr(err error) bool { + var e InvalidTransitionRandomBeaconStateMachineErr + return errors.As(err, &e) +} + +// NewInvalidTransitionRandomBeaconStateMachineErr constructs a new InvalidTransitionRandomBeaconStateMachineErr error. +func NewInvalidTransitionRandomBeaconStateMachineErr(from, to flow.DKGState) error { + return InvalidTransitionRandomBeaconStateMachineErr{ + From: from, + To: to, + } +} + +// NewInvalidTransitionRandomBeaconStateMachineErrf constructs a new InvalidTransitionRandomBeaconStateMachineErr error with a formatted message. +func NewInvalidTransitionRandomBeaconStateMachineErrf(from, to flow.DKGState, msg string, args ...any) error { + return InvalidTransitionRandomBeaconStateMachineErr{ + From: from, + To: to, + err: fmt.Errorf(msg, args...), + } +} diff --git a/storage/mock/dkg_state.go b/storage/mock/dkg_state.go index 10451250fbf..84332eb4449 100644 --- a/storage/mock/dkg_state.go +++ b/storage/mock/dkg_state.go @@ -14,23 +14,23 @@ type DKGState struct { mock.Mock } -// GetDKGEndState provides a mock function with given fields: epochCounter -func (_m *DKGState) GetDKGEndState(epochCounter uint64) (flow.DKGEndState, error) { +// GetDKGStarted provides a mock function with given fields: epochCounter +func (_m *DKGState) GetDKGStarted(epochCounter uint64) (bool, error) { ret := _m.Called(epochCounter) if len(ret) == 0 { - panic("no return value specified for GetDKGEndState") + panic("no return value specified for GetDKGStarted") } - var r0 flow.DKGEndState + var r0 bool var r1 error - if rf, ok := ret.Get(0).(func(uint64) (flow.DKGEndState, error)); ok { + if rf, ok := ret.Get(0).(func(uint64) (bool, error)); ok { return rf(epochCounter) } - if rf, ok := ret.Get(0).(func(uint64) flow.DKGEndState); ok { + if rf, ok := ret.Get(0).(func(uint64) bool); ok { r0 = rf(epochCounter) } else { - r0 = ret.Get(0).(flow.DKGEndState) + r0 = ret.Get(0).(bool) } if rf, ok := ret.Get(1).(func(uint64) error); ok { @@ -42,23 +42,23 @@ func (_m *DKGState) GetDKGEndState(epochCounter uint64) (flow.DKGEndState, error return r0, r1 } -// GetDKGStarted provides a mock function with given fields: epochCounter -func (_m *DKGState) GetDKGStarted(epochCounter uint64) (bool, error) { +// GetDKGState provides a mock function with given fields: epochCounter +func (_m *DKGState) GetDKGState(epochCounter uint64) (flow.DKGState, error) { ret := _m.Called(epochCounter) if len(ret) == 0 { - panic("no return value specified for GetDKGStarted") + panic("no return value specified for GetDKGState") } - var r0 bool + var r0 flow.DKGState var r1 error - if rf, ok := ret.Get(0).(func(uint64) (bool, error)); ok { + if rf, ok := ret.Get(0).(func(uint64) (flow.DKGState, error)); ok { return rf(epochCounter) } - if rf, ok := ret.Get(0).(func(uint64) bool); ok { + if rf, ok := ret.Get(0).(func(uint64) flow.DKGState); ok { r0 = rf(epochCounter) } else { - r0 = ret.Get(0).(bool) + r0 = ret.Get(0).(flow.DKGState) } if rf, ok := ret.Get(1).(func(uint64) error); ok { @@ -89,7 +89,7 @@ func (_m *DKGState) InsertMyBeaconPrivateKey(epochCounter uint64, key crypto.Pri } // RetrieveMyBeaconPrivateKey provides a mock function with given fields: epochCounter -func (_m *DKGState) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) { +func (_m *DKGState) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, bool, error) { ret := _m.Called(epochCounter) if len(ret) == 0 { @@ -97,8 +97,9 @@ func (_m *DKGState) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.Priv } var r0 crypto.PrivateKey - var r1 error - if rf, ok := ret.Get(0).(func(uint64) (crypto.PrivateKey, error)); ok { + var r1 bool + var r2 error + if rf, ok := ret.Get(0).(func(uint64) (crypto.PrivateKey, bool, error)); ok { return rf(epochCounter) } if rf, ok := ret.Get(0).(func(uint64) crypto.PrivateKey); ok { @@ -109,26 +110,32 @@ func (_m *DKGState) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.Priv } } - if rf, ok := ret.Get(1).(func(uint64) error); ok { + if rf, ok := ret.Get(1).(func(uint64) bool); ok { r1 = rf(epochCounter) } else { - r1 = ret.Error(1) + r1 = ret.Get(1).(bool) } - return r0, r1 + if rf, ok := ret.Get(2).(func(uint64) error); ok { + r2 = rf(epochCounter) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 } -// SetDKGEndState provides a mock function with given fields: epochCounter, endState -func (_m *DKGState) SetDKGEndState(epochCounter uint64, endState flow.DKGEndState) error { - ret := _m.Called(epochCounter, endState) +// SetDKGState provides a mock function with given fields: epochCounter, newState +func (_m *DKGState) SetDKGState(epochCounter uint64, newState flow.DKGState) error { + ret := _m.Called(epochCounter, newState) if len(ret) == 0 { - panic("no return value specified for SetDKGEndState") + panic("no return value specified for SetDKGState") } var r0 error - if rf, ok := ret.Get(0).(func(uint64, flow.DKGEndState) error); ok { - r0 = rf(epochCounter, endState) + if rf, ok := ret.Get(0).(func(uint64, flow.DKGState) error); ok { + r0 = rf(epochCounter, newState) } else { r0 = ret.Error(0) } @@ -136,22 +143,34 @@ func (_m *DKGState) SetDKGEndState(epochCounter uint64, endState flow.DKGEndStat return r0 } -// SetDKGStarted provides a mock function with given fields: epochCounter -func (_m *DKGState) SetDKGStarted(epochCounter uint64) error { +// UnsafeRetrieveMyBeaconPrivateKey provides a mock function with given fields: epochCounter +func (_m *DKGState) UnsafeRetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) { ret := _m.Called(epochCounter) if len(ret) == 0 { - panic("no return value specified for SetDKGStarted") + panic("no return value specified for UnsafeRetrieveMyBeaconPrivateKey") } - var r0 error - if rf, ok := ret.Get(0).(func(uint64) error); ok { + var r0 crypto.PrivateKey + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (crypto.PrivateKey, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) crypto.PrivateKey); ok { r0 = rf(epochCounter) } else { - r0 = ret.Error(0) + if ret.Get(0) != nil { + r0 = ret.Get(0).(crypto.PrivateKey) + } } - return r0 + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } // NewDKGState creates a new instance of DKGState. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. diff --git a/storage/mock/dkg_state_reader.go b/storage/mock/dkg_state_reader.go new file mode 100644 index 00000000000..eb62bf0ecab --- /dev/null +++ b/storage/mock/dkg_state_reader.go @@ -0,0 +1,152 @@ +// Code generated by mockery v2.43.2. DO NOT EDIT. + +package mock + +import ( + crypto "github.com/onflow/crypto" + flow "github.com/onflow/flow-go/model/flow" + + mock "github.com/stretchr/testify/mock" +) + +// DKGStateReader is an autogenerated mock type for the DKGStateReader type +type DKGStateReader struct { + mock.Mock +} + +// GetDKGStarted provides a mock function with given fields: epochCounter +func (_m *DKGStateReader) GetDKGStarted(epochCounter uint64) (bool, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for GetDKGStarted") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (bool, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) bool); ok { + r0 = rf(epochCounter) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetDKGState provides a mock function with given fields: epochCounter +func (_m *DKGStateReader) GetDKGState(epochCounter uint64) (flow.DKGState, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for GetDKGState") + } + + var r0 flow.DKGState + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (flow.DKGState, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) flow.DKGState); ok { + r0 = rf(epochCounter) + } else { + r0 = ret.Get(0).(flow.DKGState) + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// RetrieveMyBeaconPrivateKey provides a mock function with given fields: epochCounter +func (_m *DKGStateReader) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, bool, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for RetrieveMyBeaconPrivateKey") + } + + var r0 crypto.PrivateKey + var r1 bool + var r2 error + if rf, ok := ret.Get(0).(func(uint64) (crypto.PrivateKey, bool, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) crypto.PrivateKey); ok { + r0 = rf(epochCounter) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(crypto.PrivateKey) + } + } + + if rf, ok := ret.Get(1).(func(uint64) bool); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Get(1).(bool) + } + + if rf, ok := ret.Get(2).(func(uint64) error); ok { + r2 = rf(epochCounter) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 +} + +// UnsafeRetrieveMyBeaconPrivateKey provides a mock function with given fields: epochCounter +func (_m *DKGStateReader) UnsafeRetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for UnsafeRetrieveMyBeaconPrivateKey") + } + + var r0 crypto.PrivateKey + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (crypto.PrivateKey, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) crypto.PrivateKey); ok { + r0 = rf(epochCounter) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(crypto.PrivateKey) + } + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewDKGStateReader creates a new instance of DKGStateReader. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDKGStateReader(t interface { + mock.TestingT + Cleanup(func()) +}) *DKGStateReader { + mock := &DKGStateReader{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/storage/mock/epoch_recovery_my_beacon_key.go b/storage/mock/epoch_recovery_my_beacon_key.go index cb8c4720675..79c5a817a5a 100644 --- a/storage/mock/epoch_recovery_my_beacon_key.go +++ b/storage/mock/epoch_recovery_my_beacon_key.go @@ -4,6 +4,8 @@ package mock import ( crypto "github.com/onflow/crypto" + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" ) @@ -12,6 +14,62 @@ type EpochRecoveryMyBeaconKey struct { mock.Mock } +// GetDKGStarted provides a mock function with given fields: epochCounter +func (_m *EpochRecoveryMyBeaconKey) GetDKGStarted(epochCounter uint64) (bool, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for GetDKGStarted") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (bool, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) bool); ok { + r0 = rf(epochCounter) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetDKGState provides a mock function with given fields: epochCounter +func (_m *EpochRecoveryMyBeaconKey) GetDKGState(epochCounter uint64) (flow.DKGState, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for GetDKGState") + } + + var r0 flow.DKGState + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (flow.DKGState, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) flow.DKGState); ok { + r0 = rf(epochCounter) + } else { + r0 = ret.Get(0).(flow.DKGState) + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // RetrieveMyBeaconPrivateKey provides a mock function with given fields: epochCounter func (_m *EpochRecoveryMyBeaconKey) RetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, bool, error) { ret := _m.Called(epochCounter) @@ -49,6 +107,36 @@ func (_m *EpochRecoveryMyBeaconKey) RetrieveMyBeaconPrivateKey(epochCounter uint return r0, r1, r2 } +// UnsafeRetrieveMyBeaconPrivateKey provides a mock function with given fields: epochCounter +func (_m *EpochRecoveryMyBeaconKey) UnsafeRetrieveMyBeaconPrivateKey(epochCounter uint64) (crypto.PrivateKey, error) { + ret := _m.Called(epochCounter) + + if len(ret) == 0 { + panic("no return value specified for UnsafeRetrieveMyBeaconPrivateKey") + } + + var r0 crypto.PrivateKey + var r1 error + if rf, ok := ret.Get(0).(func(uint64) (crypto.PrivateKey, error)); ok { + return rf(epochCounter) + } + if rf, ok := ret.Get(0).(func(uint64) crypto.PrivateKey); ok { + r0 = rf(epochCounter) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(crypto.PrivateKey) + } + } + + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(epochCounter) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // UpsertMyBeaconPrivateKey provides a mock function with given fields: epochCounter, key func (_m *EpochRecoveryMyBeaconKey) UpsertMyBeaconPrivateKey(epochCounter uint64, key crypto.PrivateKey) error { ret := _m.Called(epochCounter, key)