diff --git a/.changelog/5354.breaking.md b/.changelog/5354.breaking.md new file mode 100644 index 00000000000..774acc8e3b2 --- /dev/null +++ b/.changelog/5354.breaking.md @@ -0,0 +1,7 @@ +go/worker/compute/executor/committee: Support backup proposers + +Starting now, all executor committee workers are permitted to schedule +transactions, each with distinct per-round priority. Priority dictates +the time after which a worker can propose a new batch. The consensus +layer tracks all published executor commitments and tries to build +a new runtime block on a proposal with the highest priority. diff --git a/go/consensus/cometbft/apps/registry/transactions_test.go b/go/consensus/cometbft/apps/registry/transactions_test.go index 4371487c248..d9716238235 100644 --- a/go/consensus/cometbft/apps/registry/transactions_test.go +++ b/go/consensus/cometbft/apps/registry/transactions_test.go @@ -648,7 +648,7 @@ func TestRegisterRuntime(t *testing.T) { BatchFlushTimeout: 100_000_000, MaxBatchSize: 100, MaxBatchSizeBytes: 100_000_000, - ProposerTimeout: 2, + SchedulerTimeout: 2, }, Deployments: []*registry.VersionInfo{ { diff --git a/go/consensus/cometbft/apps/roothash/messages.go b/go/consensus/cometbft/apps/roothash/messages.go index 7578bcf2bfc..9a9120813d5 100644 --- a/go/consensus/cometbft/apps/roothash/messages.go +++ b/go/consensus/cometbft/apps/roothash/messages.go @@ -6,7 +6,6 @@ import ( beacon "github.com/oasisprotocol/oasis-core/go/beacon/api" "github.com/oasisprotocol/oasis-core/go/common/cbor" "github.com/oasisprotocol/oasis-core/go/common/errors" - "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/api" tmapi "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/api" registryState "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/apps/registry/state" roothashApi "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/apps/roothash/api" @@ -82,7 +81,7 @@ func (app *rootHashApplication) processRuntimeMessages( return events, nil } -func (app *rootHashApplication) doBeforeSchedule(ctx *api.Context, msg interface{}) (interface{}, error) { +func (app *rootHashApplication) doBeforeSchedule(ctx *tmapi.Context, msg interface{}) (interface{}, error) { epoch := msg.(beacon.EpochTime) ctx.Logger().Debug("processing liveness statistics before scheduling", @@ -109,7 +108,7 @@ func (app *rootHashApplication) doBeforeSchedule(ctx *api.Context, msg interface return nil, nil } -func (app *rootHashApplication) changeParameters(ctx *api.Context, msg interface{}, apply bool) (interface{}, error) { +func (app *rootHashApplication) changeParameters(ctx *tmapi.Context, msg interface{}, apply bool) (interface{}, error) { // Unmarshal changes and check if they should be applied to this module. proposal, ok := msg.(*governance.ChangeParametersProposal) if !ok { diff --git a/go/consensus/cometbft/apps/roothash/roothash.go b/go/consensus/cometbft/apps/roothash/roothash.go index ed13848dafe..bfad31872df 100644 --- a/go/consensus/cometbft/apps/roothash/roothash.go +++ b/go/consensus/cometbft/apps/roothash/roothash.go @@ -286,6 +286,11 @@ func (app *rootHashApplication) prepareNewCommittees( func (app *rootHashApplication) emitEmptyBlock(ctx *tmapi.Context, runtime *roothash.RuntimeState, hdrType block.HeaderType) error { blk := block.NewEmptyBlock(runtime.CurrentBlock, uint64(ctx.Now().Unix()), hdrType) + ctx.Logger().Debug("emitting empty block", + "round", blk.Header.Round, + "type", blk.Header.HeaderType, + ) + runtime.CurrentBlock = blk runtime.CurrentBlockHeight = ctx.BlockHeight() + 1 // Current height is ctx.BlockHeight() + 1 // Do not update LastNormal{Round,Height} as empty blocks are not emitted by the runtime. @@ -541,7 +546,7 @@ func (app *rootHashApplication) tryFinalizeExecutorCommits( //nolint: gocyclo pool := rtState.ExecutorPool // Remember the index of the transaction scheduler within the committee. - schedulerIdx, err := pool.Committee.TransactionSchedulerIdx(pool.Round) + schedulerIdx, err := pool.Committee.SchedulerIdx(pool.Round) if err != nil { return err } @@ -552,16 +557,22 @@ func (app *rootHashApplication) tryFinalizeExecutorCommits( //nolint: gocyclo } livenessStats := rtState.LivenessStatistics + // Compute the priority of the pool scheduler. + // If the pool is empty, the priority will be set to infinity. + poolSchedulerID, _ := pool.Scheduler() + priority := pool.Committee.SchedulingPriority(pool.Round, poolSchedulerID) + commit, err := pool.TryFinalize(ctx.BlockHeight(), runtime.Executor.RoundTimeout, forced, true) if err == commitment.ErrDiscrepancyDetected { ctx.Logger().Warn("executor discrepancy detected", "round", round, + "priority", priority, logging.LogEvent, roothash.LogEventExecutionDiscrepancyDetected, ) ctx.EmitEvent( tmapi.NewEventBuilder(app.Name()). - TypedAttribute(&roothash.ExecutionDiscrepancyDetectedEvent{Timeout: forced}). + TypedAttribute(&roothash.ExecutionDiscrepancyDetectedEvent{Priority: priority, Timeout: forced}). TypedAttribute(&roothash.RuntimeIDAttribute{ID: runtime.ID}), ) @@ -577,11 +588,19 @@ func (app *rootHashApplication) tryFinalizeExecutorCommits( //nolint: gocyclo // Round has been finalized. ctx.Logger().Debug("finalized round", "round", round, + "priority", priority, ) - // Record that the round was finalized and that the scheduler received enough commitments. livenessStats.TotalRounds++ - livenessStats.FinalizedProposals[schedulerIdx]++ + + // Record if the transaction scheduler received enough commitments. + schedulerID := pool.Committee.Members[schedulerIdx].PublicKey + switch schedulerID.Equal(poolSchedulerID) { + case true: + livenessStats.FinalizedProposals[schedulerIdx]++ + case false: + livenessStats.MissedProposals[schedulerIdx]++ + } ec := commit.ToDDResult().(*commitment.ExecutorCommitment) diff --git a/go/consensus/cometbft/apps/roothash/transactions.go b/go/consensus/cometbft/apps/roothash/transactions.go index 958825d90d5..31dedb37180 100644 --- a/go/consensus/cometbft/apps/roothash/transactions.go +++ b/go/consensus/cometbft/apps/roothash/transactions.go @@ -74,7 +74,7 @@ func (app *rootHashApplication) executorProposerTimeout( } // Ensure enough blocks have passed since round start. - proposerTimeout := rtState.Runtime.TxnScheduler.ProposerTimeout + proposerTimeout := rtState.Runtime.TxnScheduler.SchedulerTimeout currentBlockHeight := rtState.CurrentBlockHeight if height := ctx.BlockHeight(); height < currentBlockHeight+proposerTimeout { ctx.Logger().Debug("failed requesting proposer round timeout, timeout not allowed yet", @@ -86,7 +86,7 @@ func (app *rootHashApplication) executorProposerTimeout( } // Ensure request is valid. - if err = rtState.ExecutorPool.CheckProposerTimeout(ctx, rtState.CurrentBlock, nl, ctx.TxSigner(), rpt.Round); err != nil { + if err = rtState.ExecutorPool.CheckSchedulerTimeout(ctx, rtState.CurrentBlock, nl, ctx.TxSigner(), rpt.Round); err != nil { ctx.Logger().Debug("failed requesting proposer round timeout", "err", err, "round", rtState.CurrentBlock.Header.Round, @@ -96,7 +96,7 @@ func (app *rootHashApplication) executorProposerTimeout( } // Record that the scheduler did not propose. - schedulerIdx, err := rtState.ExecutorPool.Committee.TransactionSchedulerIdx(rpt.Round) + schedulerIdx, err := rtState.ExecutorPool.Committee.SchedulerIdx(rpt.Round) if err != nil { return err } diff --git a/go/consensus/cometbft/apps/roothash/transactions_test.go b/go/consensus/cometbft/apps/roothash/transactions_test.go index 2497b8910dd..a2fe45f7d50 100644 --- a/go/consensus/cometbft/apps/roothash/transactions_test.go +++ b/go/consensus/cometbft/apps/roothash/transactions_test.go @@ -22,7 +22,6 @@ import ( schedulerState "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/apps/scheduler/state" stakingState "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/apps/staking/state" genesisTestHelpers "github.com/oasisprotocol/oasis-core/go/genesis/tests" - "github.com/oasisprotocol/oasis-core/go/governance/api" governance "github.com/oasisprotocol/oasis-core/go/governance/api" registry "github.com/oasisprotocol/oasis-core/go/registry/api" roothash "github.com/oasisprotocol/oasis-core/go/roothash/api" @@ -197,9 +196,9 @@ func TestMessagesGasEstimation(t *testing.T) { // Each update_runtime message costs 3000 gas. {Registry: &message.RegistryMessage{UpdateRuntime: ®istry.Runtime{}}}, // Each cast vote message costs 1000 gas. - {Governance: &message.GovernanceMessage{CastVote: &api.ProposalVote{}}}, + {Governance: &message.GovernanceMessage{CastVote: &governance.ProposalVote{}}}, // Each submit proposal message costs 2000 gas. - {Governance: &message.GovernanceMessage{SubmitProposal: &api.ProposalContent{}}}, + {Governance: &message.GovernanceMessage{SubmitProposal: &governance.ProposalContent{}}}, } msgsHash := message.MessagesHash(msgs) @@ -209,6 +208,7 @@ func TestMessagesGasEstimation(t *testing.T) { ec := commitment.ExecutorCommitment{ NodeID: sk.Public(), Header: commitment.ExecutorCommitmentHeader{ + SchedulerID: sk.Public(), Header: commitment.ComputeResultsHeader{ Round: newBlk.Header.Round, PreviousHash: newBlk.Header.PreviousHash, diff --git a/go/consensus/cometbft/apps/scheduler/debug_force.go b/go/consensus/cometbft/apps/scheduler/debug_force.go index 272c51b9483..18a4acfe788 100644 --- a/go/consensus/cometbft/apps/scheduler/debug_force.go +++ b/go/consensus/cometbft/apps/scheduler/debug_force.go @@ -111,14 +111,13 @@ func (app *schedulerApplication) debugForceRoles( mayBeAny []*scheduler.CommitteeNode ) - for i, n := range elected { - committeeNode := elected[i] - if ri, ok := state.params[n.PublicKey]; ok { + for _, committeeNode := range elected { + if ri, ok := state.params[committeeNode.PublicKey]; ok { if ri.IsScheduler { if mustBeScheduler != nil { ctx.Logger().Error("already have a forced scheduler", "existing", mustBeScheduler.PublicKey, - "new", n.PublicKey, + "new", committeeNode.PublicKey, ) return false, nil } diff --git a/go/genesis/genesis_test.go b/go/genesis/genesis_test.go index 474dc0062e5..b3c6afed98c 100644 --- a/go/genesis/genesis_test.go +++ b/go/genesis/genesis_test.go @@ -212,7 +212,7 @@ func TestGenesisSanityCheck(t *testing.T) { BatchFlushTimeout: 1 * time.Second, MaxBatchSize: 1, MaxBatchSizeBytes: 1024, - ProposerTimeout: 20, + SchedulerTimeout: 20, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ AnyNode: ®istry.AnyNodeRuntimeAdmissionPolicy{}, diff --git a/go/oasis-net-runner/fixtures/default.go b/go/oasis-net-runner/fixtures/default.go index e4bf26d1d32..ed09ef7cdf3 100644 --- a/go/oasis-net-runner/fixtures/default.go +++ b/go/oasis-net-runner/fixtures/default.go @@ -221,7 +221,7 @@ func newDefaultFixture() (*oasis.NetworkFixture, error) { MaxBatchSize: 1000, MaxBatchSizeBytes: 16 * 1024 * 1024, // 16 MiB BatchFlushTimeout: 1 * time.Second, - ProposerTimeout: 20, + SchedulerTimeout: 20, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ AnyNode: ®istry.AnyNodeRuntimeAdmissionPolicy{}, diff --git a/go/oasis-node/cmd/control/runtime_stats.go b/go/oasis-node/cmd/control/runtime_stats.go index 554b529e235..083ff5ff17b 100644 --- a/go/oasis-node/cmd/control/runtime_stats.go +++ b/go/oasis-node/cmd/control/runtime_stats.go @@ -495,7 +495,7 @@ func doRuntimeStats(cmd *cobra.Command, args []string) { //nolint:gocyclo } // Set committee info. currentCommittee = state.ExecutorPool.Committee - currentScheduler, err = currentCommittee.TransactionScheduler(currentRound) + currentScheduler, err = currentCommittee.Scheduler(currentRound) if err != nil { logger.Error("failed to query transaction scheduler", "err", err, diff --git a/go/oasis-node/cmd/debug/byzantine/byzantine.go b/go/oasis-node/cmd/debug/byzantine/byzantine.go index 9dfc1b08916..54584370b56 100644 --- a/go/oasis-node/cmd/debug/byzantine/byzantine.go +++ b/go/oasis-node/cmd/debug/byzantine/byzantine.go @@ -11,6 +11,7 @@ import ( "github.com/spf13/viper" "github.com/oasisprotocol/oasis-core/go/common" + "github.com/oasisprotocol/oasis-core/go/common/crypto/signature" "github.com/oasisprotocol/oasis-core/go/common/logging" "github.com/oasisprotocol/oasis-core/go/common/node" "github.com/oasisprotocol/oasis-core/go/common/sgx/ias" @@ -154,31 +155,36 @@ func doExecutorScenario(cmd *cobra.Command, args []string) { //nolint: gocyclo return } + // Get the latest roothash block. + var blk *block.Block + blk, err = getRoothashLatestBlock(ctx, b.cometbft.service, runtimeID) + if err != nil { + panic(fmt.Errorf("failed getting latest roothash block: %w", err)) + } + + var schedulerID signature.PublicKey cbc := newComputeBatchContext(b.chainContext, runtimeID) switch isTxScheduler { case true: // If we are the transaction scheduler, we wait for transactions and schedule them. var cont bool - cont, err = b.receiveAndScheduleTransactions(ctx, cbc, executorMode) + cont, err = b.receiveAndScheduleTransactions(ctx, cbc, blk, executorMode) if err != nil { - panic(fmt.Sprintf("comptue transaction scheduling failed: %+v", err)) + panic(fmt.Sprintf("compute transaction scheduling failed: %+v", err)) } if !cont { return } + + schedulerID = b.identity.NodeSigner.Public() case false: // If we are not the scheduler, receive transactions and the proposal. if err = cbc.receiveProposal(b.p2p); err != nil { panic(fmt.Sprintf("compute receive proposal failed: %+v", err)) } logger.Debug("executor: received proposal", "proposal", cbc.proposal) - } - // Get latest roothash block. - var blk *block.Block - blk, err = getRoothashLatestBlock(ctx, b.cometbft.service, runtimeID) - if err != nil { - panic(fmt.Errorf("failed getting latest roothash block: %w", err)) + schedulerID = cbc.proposal.NodeID } if err = cbc.openTrees(ctx, blk, b.storageClient); err != nil { @@ -256,11 +262,11 @@ func doExecutorScenario(cmd *cobra.Command, args []string) { //nolint: gocyclo switch executorMode { case ModeExecutorFailureIndicating: - if err = cbc.createCommitment(b.identity, b.rak, commitment.FailureUnknown); err != nil { + if err = cbc.createCommitment(b.identity, schedulerID, b.rak, commitment.FailureUnknown); err != nil { panic(fmt.Sprintf("compute create commitment failed: %+v", err)) } default: - if err = cbc.createCommitment(b.identity, b.rak, commitment.FailureNone); err != nil { + if err = cbc.createCommitment(b.identity, schedulerID, b.rak, commitment.FailureNone); err != nil { panic(fmt.Sprintf("compute create commitment failed: %+v", err)) } diff --git a/go/oasis-node/cmd/debug/byzantine/executor.go b/go/oasis-node/cmd/debug/byzantine/executor.go index 7520da1552c..692b82b163c 100644 --- a/go/oasis-node/cmd/debug/byzantine/executor.go +++ b/go/oasis-node/cmd/debug/byzantine/executor.go @@ -302,6 +302,7 @@ func (cbc *computeBatchContext) commitTrees(ctx context.Context) error { func (cbc *computeBatchContext) createCommitment( id *identity.Identity, + schedulerID signature.PublicKey, rak signature.Signer, failure commitment.ExecutorCommitmentFailure, ) error { @@ -320,7 +321,8 @@ func (cbc *computeBatchContext) createCommitment( ec := &commitment.ExecutorCommitment{ NodeID: id.NodeSigner.Public(), Header: commitment.ExecutorCommitmentHeader{ - Header: header, + SchedulerID: schedulerID, + Header: header, }, } if rak != nil { diff --git a/go/oasis-node/cmd/debug/byzantine/node.go b/go/oasis-node/cmd/debug/byzantine/node.go index fc75f4caa6b..c296c4973e0 100644 --- a/go/oasis-node/cmd/debug/byzantine/node.go +++ b/go/oasis-node/cmd/debug/byzantine/node.go @@ -58,35 +58,30 @@ func (b *byzantine) stop() error { return nil } -func (b *byzantine) receiveAndScheduleTransactions(ctx context.Context, cbc *computeBatchContext, mode ExecutorMode) (bool, error) { +func (b *byzantine) receiveAndScheduleTransactions(ctx context.Context, cbc *computeBatchContext, block *block.Block, mode ExecutorMode) (bool, error) { // Receive transactions. txs := cbc.receiveTransactions(b.p2p, time.Second) logger.Debug("executor: received transactions", "transactions", txs) - // Get latest roothash block. - var block *block.Block - block, err := getRoothashLatestBlock(ctx, b.cometbft.service, b.runtimeID) - if err != nil { - return false, fmt.Errorf("failed getting latest roothash block: %w", err) - } // Include transactions that nobody else has when configured to do so. if viper.GetBool(CfgExecutorProposeBogusTx) { logger.Debug("executor scheduler: including bogus transactions") - txs = append(txs, []byte("this is a bogus transction nr. 1")) + txs = append(txs, []byte("this is a bogus transition nr. 1")) } // Prepare proposal. - if err = cbc.prepareProposal(ctx, block, txs, b.identity); err != nil { + if err := cbc.prepareProposal(ctx, block, txs, b.identity); err != nil { panic(fmt.Sprintf("executor proposing batch: %+v", err)) } if mode == ModeExecutorFailureIndicating { // Submit failure indicating commitment and stop. logger.Debug("executor failure indicating: submitting commitment and stopping") - if err = cbc.createCommitment(b.identity, nil, commitment.FailureUnknown); err != nil { + schedulerID := b.identity.NodeSigner.Public() + if err := cbc.createCommitment(b.identity, schedulerID, nil, commitment.FailureUnknown); err != nil { panic(fmt.Sprintf("compute create failure indicating commitment failed: %+v", err)) } - if err = cbc.publishToChain(b.cometbft.service, b.identity); err != nil { + if err := cbc.publishToChain(b.cometbft.service, b.identity); err != nil { panic(fmt.Sprintf("compute publish to chain failed: %+v", err)) } return false, nil @@ -227,8 +222,8 @@ func initializeAndRegisterByzantineNode( b.logger.Debug("executor schedule ok") // Ensure we have the expected executor transaction scheduler role. - isTxScheduler := schedulerCheckTxScheduler(b.executorCommittee, b.identity.NodeSigner.Public(), 0) - if shouldBeExecutorProposer != isTxScheduler { + isScheduler := schedulerCheckScheduler(b.executorCommittee, b.identity.NodeSigner.Public(), 0) + if shouldBeExecutorProposer != isScheduler { return nil, fmt.Errorf("not in expected executor transaction scheduler role") } b.logger.Debug("executor tx scheduler role ok") diff --git a/go/oasis-node/cmd/debug/byzantine/scheduler.go b/go/oasis-node/cmd/debug/byzantine/scheduler.go index 1fec3c9b573..34967b841b6 100644 --- a/go/oasis-node/cmd/debug/byzantine/scheduler.go +++ b/go/oasis-node/cmd/debug/byzantine/scheduler.go @@ -75,8 +75,8 @@ func schedulerCheckScheduled(committee *scheduler.Committee, nodeID signature.Pu return fmt.Errorf("we're not scheduled") } -func schedulerCheckTxScheduler(committee *scheduler.Committee, nodeID signature.PublicKey, round uint64) bool { - scheduler, err := committee.TransactionScheduler(round) +func schedulerCheckScheduler(committee *scheduler.Committee, nodeID signature.PublicKey, round uint64) bool { + scheduler, err := committee.Scheduler(round) if err != nil { panic(err) } diff --git a/go/oasis-node/cmd/debug/txsource/workload/registration.go b/go/oasis-node/cmd/debug/txsource/workload/registration.go index a96c6ae53f0..b123c331888 100644 --- a/go/oasis-node/cmd/debug/txsource/workload/registration.go +++ b/go/oasis-node/cmd/debug/txsource/workload/registration.go @@ -63,7 +63,7 @@ func getRuntime(entityID signature.PublicKey, id common.Namespace, epoch beacon. BatchFlushTimeout: 1 * time.Second, MaxBatchSize: 1, MaxBatchSizeBytes: 1024, - ProposerTimeout: 5, + SchedulerTimeout: 5, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ AnyNode: ®istry.AnyNodeRuntimeAdmissionPolicy{}, diff --git a/go/oasis-node/node_test.go b/go/oasis-node/node_test.go index 0161375952a..24498269cb8 100644 --- a/go/oasis-node/node_test.go +++ b/go/oasis-node/node_test.go @@ -76,7 +76,7 @@ var ( MaxBatchSize: 1, MaxBatchSizeBytes: 1024, BatchFlushTimeout: 20 * time.Second, - ProposerTimeout: 20, + SchedulerTimeout: 20, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ AnyNode: ®istry.AnyNodeRuntimeAdmissionPolicy{}, diff --git a/go/oasis-test-runner/scenario/e2e/registry_cli.go b/go/oasis-test-runner/scenario/e2e/registry_cli.go index 8c0733cc100..61333507a2f 100644 --- a/go/oasis-test-runner/scenario/e2e/registry_cli.go +++ b/go/oasis-test-runner/scenario/e2e/registry_cli.go @@ -617,7 +617,7 @@ func (sc *registryCLIImpl) testRuntime(ctx context.Context, childEnv *env.Env, c BatchFlushTimeout: 11 * time.Second, MaxBatchSize: 12, MaxBatchSizeBytes: 1024, - ProposerTimeout: 5, + SchedulerTimeout: 5, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ EntityWhitelist: ®istry.EntityWhitelistRuntimeAdmissionPolicy{ diff --git a/go/oasis-test-runner/scenario/e2e/runtime/byzantine.go b/go/oasis-test-runner/scenario/e2e/runtime/byzantine.go index b6c9fd304f7..80cdce6f9ab 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/byzantine.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/byzantine.go @@ -202,9 +202,11 @@ var ( "executor-scheduler-straggler", "executor", []log.WatcherHandlerFactory{ - // Scheduler straggler should trigger round failure in first round (proposer timeout). - // In round two timeout and discrepancy detection should be triggered. - oasis.LogAssertRoundFailures(), + // The scheduler straggler should not trigger a round failure in the third round, + // as the backup scheduler should propose before other nodes trigger the timeout. + // However, the consensus timeout and discrepancy detection should be triggered + // since we don't allow any stragglers. + oasis.LogAssertNoRoundFailures(), oasis.LogAssertTimeouts(), oasis.LogAssertExecutionDiscrepancyDetected(), }, diff --git a/go/oasis-test-runner/scenario/e2e/runtime/helpers_cosnensus.go b/go/oasis-test-runner/scenario/e2e/runtime/helpers_cosnensus.go index 1f46789a70b..8c084baa3c2 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/helpers_cosnensus.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/helpers_cosnensus.go @@ -112,7 +112,7 @@ func (sc *Scenario) initialEpochTransitionsWith(ctx context.Context, fixture *oa } } if !sc.debugNoRandomInitialEpoch { - // To prevent people from writing tests that depend on very precicse + // To prevent people from writing tests that depend on very precise // timekeeping by epoch, randomize the start epoch slightly. // // If this causes your test to fail, it is not this code that is diff --git a/go/oasis-test-runner/scenario/e2e/runtime/multiple_runtimes.go b/go/oasis-test-runner/scenario/e2e/runtime/multiple_runtimes.go index f2c2376645f..0e52583da3c 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/multiple_runtimes.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/multiple_runtimes.go @@ -94,7 +94,7 @@ func (sc *multipleRuntimesImpl) Fixture() (*oasis.NetworkFixture, error) { MaxBatchSize: 100, MaxBatchSizeBytes: 1024 * 1024, BatchFlushTimeout: 1 * time.Second, - ProposerTimeout: 10, + SchedulerTimeout: 10, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ AnyNode: ®istry.AnyNodeRuntimeAdmissionPolicy{}, diff --git a/go/oasis-test-runner/scenario/e2e/runtime/runtime_governance.go b/go/oasis-test-runner/scenario/e2e/runtime/runtime_governance.go index efa4cdb53b2..7c2c3190372 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/runtime_governance.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/runtime_governance.go @@ -96,7 +96,7 @@ func (sc *runtimeGovernanceImpl) Fixture() (*oasis.NetworkFixture, error) { MaxBatchSize: 100, MaxBatchSizeBytes: 1024 * 1024, BatchFlushTimeout: 1 * time.Second, - ProposerTimeout: 10, + SchedulerTimeout: 10, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ AnyNode: ®istry.AnyNodeRuntimeAdmissionPolicy{}, @@ -344,7 +344,8 @@ func (sc *runtimeGovernanceImpl) Run(ctx context.Context, childEnv *env.Env) err return err } - // Wait for next round. + // The bogus update should cause the runtime to panic, which will result + // in no more blocks being produced. for { select { case blk := <-blkCh: @@ -352,8 +353,8 @@ func (sc *runtimeGovernanceImpl) Run(ctx context.Context, childEnv *env.Env) err if blk.Block.Header.Round <= meta.Round { continue } + return fmt.Errorf("unexpected block, the bogus update should cause the runtime to panic") case <-time.After(waitTimeout): - return fmt.Errorf("timed out waiting for runtime rounds") } break diff --git a/go/oasis-test-runner/scenario/e2e/runtime/scenario.go b/go/oasis-test-runner/scenario/e2e/runtime/scenario.go index b558ee99698..c3faa6b1417 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/scenario.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/scenario.go @@ -194,7 +194,7 @@ func (sc *Scenario) Fixture() (*oasis.NetworkFixture, error) { MaxBatchSize: 100, MaxBatchSizeBytes: 1024 * 1024, BatchFlushTimeout: 1 * time.Second, - ProposerTimeout: 20, + SchedulerTimeout: 20, MaxInMessages: 128, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ diff --git a/go/oasis-test-runner/scenario/e2e/runtime/txsource.go b/go/oasis-test-runner/scenario/e2e/runtime/txsource.go index ce06aec9d9b..1074be2d233 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/txsource.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/txsource.go @@ -448,7 +448,7 @@ func (sc *txSourceImpl) Fixture() (*oasis.NetworkFixture, error) { f.Runtimes[1].Executor.AllowedStragglers = 1 // Lower proposer and round timeouts as nodes are expected to go offline for longer time. - f.Runtimes[1].TxnScheduler.ProposerTimeout = 4 + f.Runtimes[1].TxnScheduler.SchedulerTimeout = 4 f.Runtimes[1].Executor.RoundTimeout = 10 } diff --git a/go/p2p/api/convert.go b/go/p2p/api/convert.go index 5e629e676d4..db54ec3e6a1 100644 --- a/go/p2p/api/convert.go +++ b/go/p2p/api/convert.go @@ -27,7 +27,7 @@ func PublicKeyToPeerID(pk signature.PublicKey) (core.PeerID, error) { } // PublicKeyMapToPeerIDs converts a map of public keys to a list of peer identifiers. -func PublicKeyMapToPeerIDs(pks map[signature.PublicKey]bool) ([]core.PeerID, error) { +func PublicKeyMapToPeerIDs(pks map[signature.PublicKey]struct{}) ([]core.PeerID, error) { ids := make([]core.PeerID, 0, len(pks)) for pk := range pks { id, err := PublicKeyToPeerID(pk) diff --git a/go/registry/api/runtime.go b/go/registry/api/runtime.go index 891aac6eb89..4a8ec1244ba 100644 --- a/go/registry/api/runtime.go +++ b/go/registry/api/runtime.go @@ -150,9 +150,9 @@ type TxnSchedulerParameters struct { // MaxInMessages specifies the maximum size of the incoming message queue. MaxInMessages uint32 `json:"max_in_messages,omitempty"` - // ProposerTimeout denotes the timeout (in consensus blocks) for scheduler + // SchedulerTimeout denotes the timeout (in consensus blocks) for scheduler // to propose a batch. - ProposerTimeout int64 `json:"propose_batch_timeout"` + SchedulerTimeout int64 `json:"propose_batch_timeout"` } // ValidateBasic performs basic transaction scheduler parameter validity checks. @@ -167,8 +167,8 @@ func (t *TxnSchedulerParameters) ValidateBasic() error { if t.MaxBatchSizeBytes < 1024 { return fmt.Errorf("transaction scheduler max batch bytes size parameter too small") } - if t.ProposerTimeout < 2 { - return fmt.Errorf("transaction scheduler proposer timeout parameter too small") + if t.SchedulerTimeout < 2 { + return fmt.Errorf("transaction scheduler propose timeout parameter too small") } return nil diff --git a/go/registry/api/runtime_test.go b/go/registry/api/runtime_test.go index d1d7c693d02..047f458a8b2 100644 --- a/go/registry/api/runtime_test.go +++ b/go/registry/api/runtime_test.go @@ -105,7 +105,7 @@ func TestRuntimeSerialization(t *testing.T) { MaxBatchSize: 10_000, MaxBatchSizeBytes: 10_000_000, MaxInMessages: 32, - ProposerTimeout: 1, + SchedulerTimeout: 1, }, Storage: StorageParameters{ CheckpointInterval: 33, @@ -210,7 +210,7 @@ func TestVerifyRuntime(t *testing.T) { MaxBatchSize: 10_000, MaxBatchSizeBytes: 10_000_000, MaxInMessages: 32, - ProposerTimeout: 2, + SchedulerTimeout: 2, }, Storage: StorageParameters{ CheckpointInterval: 33, @@ -293,7 +293,7 @@ func TestVerifyRuntime(t *testing.T) { MaxBatchSize: 10_000, MaxBatchSizeBytes: 10_000_000, MaxInMessages: 32, - ProposerTimeout: 2, + SchedulerTimeout: 2, }, Storage: StorageParameters{ CheckpointInterval: 33, @@ -392,7 +392,7 @@ func TestVerifyRuntime(t *testing.T) { MaxBatchSize: 10_000, MaxBatchSizeBytes: 10_000_000, MaxInMessages: 32, - ProposerTimeout: 2, + SchedulerTimeout: 2, }, Storage: StorageParameters{ CheckpointInterval: 33, @@ -492,7 +492,7 @@ func TestVerifyRuntime(t *testing.T) { MaxBatchSize: 10_000, MaxBatchSizeBytes: 10_000_000, MaxInMessages: 32, - ProposerTimeout: 2, + SchedulerTimeout: 2, }, Storage: StorageParameters{ CheckpointInterval: 33, @@ -595,7 +595,7 @@ func TestVerifyRuntime(t *testing.T) { MaxBatchSize: 10_000, MaxBatchSizeBytes: 10_000_000, MaxInMessages: 32, - ProposerTimeout: 2, + SchedulerTimeout: 2, }, Storage: StorageParameters{ CheckpointInterval: 33, diff --git a/go/registry/tests/tester.go b/go/registry/tests/tester.go index 3fd2e533e1d..26804ed1322 100644 --- a/go/registry/tests/tester.go +++ b/go/registry/tests/tester.go @@ -980,7 +980,7 @@ func testRegistryRuntime(t *testing.T, backend api.Backend, consensus consensusA re.MustNotRegister(t, backend, consensus) // Any updates to runtime parameters should fail for runtime-governed runtimes. re.Runtime.GovernanceModel = api.GovernanceRuntime - re.Runtime.TxnScheduler.ProposerTimeout = 6 + re.Runtime.TxnScheduler.SchedulerTimeout = 6 re.MustNotRegister(t, backend, consensus) re, err = NewTestRuntime([]byte("Runtime re-registration test 2"), entity, true) @@ -1835,7 +1835,7 @@ func NewTestRuntime(seed []byte, ent *TestEntity, isKeyManager bool) (*TestRunti BatchFlushTimeout: 20 * time.Second, MaxBatchSize: 1, MaxBatchSizeBytes: 1024, - ProposerTimeout: 5, + SchedulerTimeout: 5, }, AdmissionPolicy: api.RuntimeAdmissionPolicy{ AnyNode: &api.AnyNodeRuntimeAdmissionPolicy{}, diff --git a/go/roothash/api/api.go b/go/roothash/api/api.go index 26cea2a0e30..78920a2005f 100644 --- a/go/roothash/api/api.go +++ b/go/roothash/api/api.go @@ -428,6 +428,8 @@ func (e *ExecutorCommittedEvent) EventKind() string { // ExecutionDiscrepancyDetectedEvent is an execute discrepancy detected event. type ExecutionDiscrepancyDetectedEvent struct { + // Priority is the priority of the transaction scheduler. + Priority uint64 `json:"priority"` // Timeout signals whether the discrepancy was due to a timeout. Timeout bool `json:"timeout"` } diff --git a/go/roothash/api/commitment/executor.go b/go/roothash/api/commitment/executor.go index de8816eda19..035d190ac4c 100644 --- a/go/roothash/api/commitment/executor.go +++ b/go/roothash/api/commitment/executor.go @@ -86,6 +86,10 @@ const ( // ExecutorCommitmentHeader is the header of an executor commitment. type ExecutorCommitmentHeader struct { + // SchedulerID is the public key of the node that scheduled transactions + // and prepared the proposal. + SchedulerID signature.PublicKey `json:"scheduler_id"` + // Header is the compute results header. Header ComputeResultsHeader `json:"header"` diff --git a/go/roothash/api/commitment/pool.go b/go/roothash/api/commitment/pool.go index 040e9e22fd1..1437b0e7db5 100644 --- a/go/roothash/api/commitment/pool.go +++ b/go/roothash/api/commitment/pool.go @@ -35,10 +35,9 @@ var ( ErrInvalidMessages = p2pError.Permanent(errors.New(moduleName, 13, "roothash/commitment: invalid messages")) // Error code 14 is reserved for future use. ErrTimeoutNotCorrectRound = errors.New(moduleName, 15, "roothash/commitment: timeout not for correct round") - ErrNodeIsScheduler = errors.New(moduleName, 16, "roothash/commitment: node is scheduler") - ErrInvalidRound = errors.New(moduleName, 17, "roothash/commitment: invalid round") - ErrNoProposerCommitment = errors.New(moduleName, 18, "roothash/commitment: no proposer commitment") - ErrBadProposerCommitment = errors.New(moduleName, 19, "roothash/commitment: bad proposer commitment") + ErrInvalidRound = errors.New(moduleName, 16, "roothash/commitment: invalid round") + ErrNoSchedulerCommitment = errors.New(moduleName, 17, "roothash/commitment: no scheduler commitment") + ErrBadSchedulerCommitment = errors.New(moduleName, 18, "roothash/commitment: bad scheduler commitment") ) const ( @@ -88,11 +87,11 @@ type Pool struct { // memberSet is a cached committee member set. It will be automatically // constructed based on the passed Committee. - memberSet map[signature.PublicKey]bool + memberSet map[signature.PublicKey]struct{} // workerSet is a cached committee worker set. It will be automatically // constructed based on the passed Committee. - workerSet map[signature.PublicKey]bool + workerSet map[signature.PublicKey]struct{} } func (p *Pool) computeMemberSets() { @@ -100,12 +99,12 @@ func (p *Pool) computeMemberSets() { return } - p.memberSet = make(map[signature.PublicKey]bool, len(p.Committee.Members)) - p.workerSet = make(map[signature.PublicKey]bool) + p.memberSet = make(map[signature.PublicKey]struct{}, len(p.Committee.Members)) + p.workerSet = make(map[signature.PublicKey]struct{}) for _, m := range p.Committee.Members { - p.memberSet[m.PublicKey] = true + p.memberSet[m.PublicKey] = struct{}{} if m.Role == scheduler.RoleWorker { - p.workerSet[m.PublicKey] = true + p.workerSet[m.PublicKey] = struct{}{} } } } @@ -119,7 +118,8 @@ func (p *Pool) isMember(id signature.PublicKey) bool { p.computeMemberSets() } - return p.memberSet[id] + _, ok := p.memberSet[id] + return ok } func (p *Pool) isWorker(id signature.PublicKey) bool { @@ -131,19 +131,17 @@ func (p *Pool) isWorker(id signature.PublicKey) bool { p.computeMemberSets() } - return p.workerSet[id] + _, ok := p.workerSet[id] + return ok } -func (p *Pool) isScheduler(id signature.PublicKey) bool { - if p.Committee == nil { - return false +// Scheduler returns the transaction scheduler that prepared a proposal upon which +// all commitments in the pool are based. +func (p *Pool) Scheduler() (signature.PublicKey, bool) { + for _, commitment := range p.ExecuteCommitments { + return commitment.Header.SchedulerID, true } - scheduler, err := p.Committee.TransactionScheduler(p.Round) - if err != nil { - return false - } - - return scheduler.PublicKey.Equal(id) + return signature.PublicKey{}, false } // ResetCommitments resets the commitments in the pool, clears the discrepancy flag and the next @@ -157,6 +155,48 @@ func (p *Pool) ResetCommitments(round uint64) { p.NextTimeout = TimeoutNever } +func (p *Pool) prioritizeCommitments(commit *ExecutorCommitment) error { + // Prioritization is not necessary if there are no commitments in the pool + // or if the scheduler matches. + schedulerID, ok := p.Scheduler() + if !ok || schedulerID.Equal(commit.Header.SchedulerID) { + return nil + } + + // Prevent placing commitments for different proposals during discrepancy resolution. + if p.Discrepancy { + logger.Debug("executor commitment with different scheduler cannot be placed during discrepancy resolution", + "round", p.Round, + "pool_scheduler_id", schedulerID, + "commit_scheduler_id", commit.Header.SchedulerID, + ) + return ErrBadExecutorCommitment + } + + // Reject commitments for proposals with lower priority. + poolPriority := p.Committee.SchedulingPriority(p.Round, schedulerID) + commitPriority := p.Committee.SchedulingPriority(p.Round, commit.Header.SchedulerID) + + if poolPriority < commitPriority { + logger.Debug("executor commitment's priority too low", + "round", p.Round, + "pool_priority", poolPriority, + "commit_priority", commitPriority, + ) + return ErrBadExecutorCommitment + } + + // Drop commitments with lower priority and start building from scratch. + logger.Debug("executor commitment with higher priority received, dropping commitments", + "round", p.Round, + "pool_priority", poolPriority, + "commit_priority", commitPriority, + ) + p.ExecuteCommitments = nil + + return nil +} + func (p *Pool) addVerifiedExecutorCommitment( // nolint: gocyclo ctx context.Context, blk *block.Block, @@ -179,9 +219,14 @@ func (p *Pool) addVerifiedExecutorCommitment( // nolint: gocyclo return ErrNotInCommittee } - // Ensure the node did not already submit a commitment. - if _, ok := p.ExecuteCommitments[commit.NodeID]; ok { - return ErrAlreadyCommitted + // Ensure that the scheduler is allowed to schedule transactions. + // While all workers are allowed to propose, the priorities of their proposals differ. + if !p.isWorker(commit.Header.SchedulerID) { + logger.Debug("executor commitment scheduler is not in the committee", + "node_id", commit.NodeID, + "scheduler_id", commit.Header.SchedulerID, + ) + return ErrBadExecutorCommitment } if p.Runtime == nil { @@ -266,7 +311,7 @@ func (p *Pool) addVerifiedExecutorCommitment( // nolint: gocyclo } // Check emitted runtime messages. - switch p.isScheduler(commit.NodeID) { + switch commit.NodeID.Equal(commit.Header.SchedulerID) { case true: // The transaction scheduler can include messages. if uint32(len(commit.Messages)) > p.Runtime.Executor.MaxMessages { @@ -309,6 +354,17 @@ func (p *Pool) addVerifiedExecutorCommitment( // nolint: gocyclo } } + // Prioritize building on proposals with higher priority after verifying the commitment. + // This prevents resetting the pool when receiving an invalid commitment with higher priority. + if err := p.prioritizeCommitments(commit); err != nil { + return err + } + + // Ensure that the node has not already submitted a commitment. + if _, ok := p.ExecuteCommitments[commit.NodeID]; ok { + return ErrAlreadyCommitted + } + if p.ExecuteCommitments == nil { p.ExecuteCommitments = make(map[signature.PublicKey]*ExecutorCommitment) } @@ -339,7 +395,7 @@ func (p *Pool) AddExecutorCommitment( // ProcessCommitments performs a single round of commitment checks. If there are enough commitments // in the pool, it performs discrepancy detection or resolution. -func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { +func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { // nolint: gocyclo switch { case p.Committee == nil: return nil, ErrNoCommittee @@ -354,6 +410,16 @@ func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { var total, commits, failures int + // Noting to do if there are no proposals. + schedulerID, exists := p.Scheduler() + switch { + case !exists && didTimeout: + return nil, ErrNoSchedulerCommitment + case !exists: + return nil, ErrStillWaiting + } + priority := p.Committee.SchedulingPriority(p.Round, schedulerID) + // Gather votes. votes := make(map[hash.Hash]*vote) for _, n := range p.Committee.Members { @@ -387,22 +453,19 @@ func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { } // As soon as there is a discrepancy we can proceed with discrepancy resolution. - // No need to wait for all commits. - if !p.Discrepancy && len(votes) > 1 { + // No need to wait for all commits. Early transition is not possible for the backup + // proposals, as backup schedulers with lower priority could bypass the priority queue. + if !p.Discrepancy && len(votes) > 1 && priority == 0 { p.Discrepancy = true return nil, ErrDiscrepancyDetected } } - // Determine whether the proposer has submitted a commitment. - proposer, err := p.Committee.TransactionScheduler(p.Round) - if err != nil { - return nil, ErrNoCommittee - } - proposerCommit, ok := p.ExecuteCommitments[proposer.PublicKey] + // Determine whether the scheduler has submitted a commitment. + schedulerCommit, ok := p.ExecuteCommitments[schedulerID] if !ok && didTimeout { // TODO: Consider slashing for this offense. - return nil, ErrNoProposerCommitment + return nil, ErrNoSchedulerCommitment } switch p.Discrepancy { @@ -413,7 +476,7 @@ func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { // If it is already known that the number of valid commitments will not exceed the required // threshold, there is no need to wait for the timer to expire. Instead, proceed directly to // the discrepancy resolution mode, regardless of any additional commits. - if failures > allowedStragglers { + if failures > allowedStragglers && priority == 0 { p.Discrepancy = true return nil, ErrDiscrepancyDetected } @@ -428,7 +491,7 @@ func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { } // Check if the majority has been reached. - if commits < required || proposerCommit == nil { + if commits < required || schedulerCommit == nil { return nil, ErrStillWaiting } @@ -452,25 +515,25 @@ func (p *Pool) ProcessCommitments(didTimeout bool) (OpenCommitment, error) { } // Check if the majority has been reached. - if topVote.tally < required || proposerCommit == nil { + if topVote.tally < required || schedulerCommit == nil { if didTimeout { return nil, ErrInsufficientVotes } return nil, ErrStillWaiting } - // Make sure that the majority commitment is the same as the proposer commitment. - if !proposerCommit.MostlyEqual(topVote.commit) { - return nil, ErrBadProposerCommitment + // Make sure that the majority commitment is the same as the scheduler commitment. + if !schedulerCommit.MostlyEqual(topVote.commit) { + return nil, ErrBadSchedulerCommitment } } - // We must return the proposer commitment as that one contains additional data. - return proposerCommit, nil + // We must return the scheduler commitment as that one contains additional data. + return schedulerCommit, nil } -// CheckProposerTimeout verifies executor timeout request conditions. -func (p *Pool) CheckProposerTimeout( +// CheckSchedulerTimeout verifies executor timeout request conditions. +func (p *Pool) CheckSchedulerTimeout( ctx context.Context, block *block.Block, nl NodeLookup, @@ -500,12 +563,6 @@ func (p *Pool) CheckProposerTimeout( return ErrNotInCommittee } - // Ensure that the node requesting a timeout is not the scheduler for - // current round. - if p.isScheduler(id) { - return ErrNodeIsScheduler - } - return nil } diff --git a/go/roothash/api/commitment/pool_test.go b/go/roothash/api/commitment/pool_test.go index f49dd20c8a8..9fef36c4279 100644 --- a/go/roothash/api/commitment/pool_test.go +++ b/go/roothash/api/commitment/pool_test.go @@ -65,7 +65,7 @@ func TestPoolDefault(t *testing.T) { _, err = pool.ProcessCommitments(false) require.Error(t, err, "ProcessCommitments") require.Equal(t, ErrNoCommittee, err) - err = pool.CheckProposerTimeout(context.Background(), blk, &staticNodeLookup{}, sk.Public(), 0) + err = pool.CheckSchedulerTimeout(context.Background(), blk, &staticNodeLookup{}, sk.Public(), 0) require.Error(t, err, "CheckProposerTimeout") require.Equal(t, ErrNoCommittee, err) } @@ -111,7 +111,7 @@ func TestPoolSingleCommitment(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) nl := &staticNodeLookup{ runtime: &node.Runtime{ @@ -133,7 +133,7 @@ func TestPoolSingleCommitment(t *testing.T) { {"MissingInMessagesHash", func(ec *ExecutorCommitment) { ec.Header.Header.InMessagesHash = nil }, ErrBadExecutorCommitment}, {"BadFailureIndicating", func(ec *ExecutorCommitment) { ec.Header.Failure = FailureUnknown }, ErrBadExecutorCommitment}, } { - _, _, invalidEc := generateExecutorCommitment(t, pool.Round) + _, _, invalidEc := generateExecutorCommitment(t, pool.Round, committee) tc.fn(&invalidEc) @@ -157,7 +157,7 @@ func TestPoolSingleCommitment(t *testing.T) { require.Equal(t, ErrStillWaiting, err, "ProcessCommitments") _, err = pool.ProcessCommitments(true) require.Error(t, err, "ProcessCommitments") - require.Equal(t, ErrNoProposerCommitment, err, "ProcessCommitments") + require.Equal(t, ErrNoSchedulerCommitment, err, "ProcessCommitments") // Test message validator function. ecWithMsgs := ec @@ -251,7 +251,7 @@ func TestPoolSingleCommitmentTEE(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) rakSig, err := signature.Sign(skRAK, ComputeResultsHeaderSignatureContext, cbor.Marshal(ec.Header.Header)) require.NoError(t, err, "Sign") ec.Header.RAKSignature = &rakSig.Signature @@ -266,7 +266,7 @@ func TestPoolSingleCommitmentTEE(t *testing.T) { require.Equal(t, ErrStillWaiting, err, "ProcessCommitments") _, err = pool.ProcessCommitments(true) require.Error(t, err, "ProcessCommitments") - require.Equal(t, ErrNoProposerCommitment, err, "ProcessCommitments") + require.Equal(t, ErrNoSchedulerCommitment, err, "ProcessCommitments") // Adding a commitment should succeed. err = pool.AddExecutorCommitment(context.Background(), childBlk, nl, &ec, nil) @@ -309,7 +309,7 @@ func TestPoolStragglers(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -331,7 +331,7 @@ func TestPoolStragglers(t *testing.T) { require.Equal(t, ErrStillWaiting, err, "ProcessCommitments") _, err = pool.ProcessCommitments(true) require.Error(t, err, "ProcessCommitments") - require.Equal(t, ErrNoProposerCommitment, err, "ProcessCommitments") + require.Equal(t, ErrNoSchedulerCommitment, err, "ProcessCommitments") // Adding commitment 1 should succeed. err = pool.AddExecutorCommitment(context.Background(), childBlk, nl, &ec1, nil) @@ -354,7 +354,7 @@ func TestPoolStragglers(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec.NodeID = sk1.Public() err := ec.Sign(sk1, rt.ID) @@ -387,7 +387,7 @@ func TestPoolStragglers(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -430,7 +430,7 @@ func TestPoolStragglers(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -486,7 +486,7 @@ func TestPoolStragglers(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -555,7 +555,7 @@ func TestPoolTwoCommitments(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -640,7 +640,7 @@ func TestPoolTwoCommitments(t *testing.T) { dc, err := pool.ProcessCommitments(false) require.Nil(t, dc, "ProcessCommitments") require.Error(t, err, "ProcessCommitments") - require.Equal(t, ErrBadProposerCommitment, err) + require.Equal(t, ErrBadSchedulerCommitment, err) }) t.Run("DiscrepancyResolutionRoleOverlap", func(t *testing.T) { @@ -773,7 +773,7 @@ func TestPoolManyCommitments(t *testing.T) { Round: 0, } - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -805,7 +805,7 @@ func TestPoolManyCommitments(t *testing.T) { Round: 0, } - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec2 := ec ec2.NodeID = sk2.Public() @@ -838,7 +838,7 @@ func TestPoolManyCommitments(t *testing.T) { Discrepancy: true, } - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec4 := ec ec4.NodeID = sk4.Public() @@ -872,7 +872,7 @@ func TestPoolManyCommitments(t *testing.T) { Discrepancy: true, } - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec4 := ec ec4.NodeID = sk4.Public() @@ -926,17 +926,19 @@ func TestPoolFailureIndicatingCommitment(t *testing.T) { } // Generate an executor commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) // Generate a valid commitment. ec1 := ec ec1.NodeID = sk1.Public() + ec1.Header.SchedulerID = sk1.Public() err := ec1.Sign(sk1, rt.ID) require.NoError(t, err, "ec1.Sign") failedEc := ExecutorCommitment{ NodeID: sk2.Public(), Header: ExecutorCommitmentHeader{ + SchedulerID: sk1.Public(), Header: ComputeResultsHeader{ Round: ec.Header.Header.Round, PreviousHash: ec.Header.Header.PreviousHash, @@ -949,6 +951,7 @@ func TestPoolFailureIndicatingCommitment(t *testing.T) { ec3 := ec ec3.NodeID = sk3.Public() + ec3.Header.SchedulerID = sk1.Public() err = ec3.Sign(sk3, rt.ID) require.NoError(t, err, "ec3.Sign") @@ -958,7 +961,7 @@ func TestPoolFailureIndicatingCommitment(t *testing.T) { require.Equal(t, ErrStillWaiting, err, "ProcessCommitments") _, err = pool.ProcessCommitments(true) require.Error(t, err, "ProcessCommitments") - require.Equal(t, ErrNoProposerCommitment, err, "ProcessCommitments") + require.Equal(t, ErrNoSchedulerCommitment, err, "ProcessCommitments") // Adding a commitment should succeed. err = pool.AddExecutorCommitment(context.Background(), childBlk, nl, &ec1, nil) @@ -1010,17 +1013,19 @@ func TestPoolFailureIndicatingCommitment(t *testing.T) { } // Generate an executor commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) // Generate a valid commitment. ec1 := ec ec1.NodeID = sk1.Public() + ec1.Header.SchedulerID = sk1.Public() err := ec1.Sign(sk1, rt.ID) require.NoError(t, err, "ec1.Sign") failedEc := ExecutorCommitment{ NodeID: sk2.Public(), Header: ExecutorCommitmentHeader{ + SchedulerID: sk1.Public(), Header: ComputeResultsHeader{ Round: ec.Header.Header.Round, PreviousHash: ec.Header.Header.PreviousHash, @@ -1033,6 +1038,7 @@ func TestPoolFailureIndicatingCommitment(t *testing.T) { ec3 := ec ec3.NodeID = sk3.Public() + ec3.Header.SchedulerID = sk1.Public() err = ec3.Sign(sk3, rt.ID) require.NoError(t, err, "ec3.Sign") @@ -1073,6 +1079,7 @@ func TestPoolFailureIndicatingCommitment(t *testing.T) { ec3.Header.SetFailure(FailureUnknown) ec3.NodeID = sk3.Public() + ec3.Header.SchedulerID = sk1.Public() err := ec3.Sign(sk3, rt.ID) require.NoError(t, err, "SignExecutorCommitment") @@ -1132,7 +1139,7 @@ func TestPoolSerialization(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec.NodeID = sk.Public() err = ec.Sign(sk, rt.ID) @@ -1175,7 +1182,7 @@ func TestTryFinalize(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -1235,7 +1242,7 @@ func TestTryFinalize(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() @@ -1318,12 +1325,6 @@ func TestExecutorTimeoutRequest(t *testing.T) { expectedError error } for _, tc := range []*testCase{ - // Scheduler (sk1 at round 0), is not allowed to request a timeout. - { - signer: sk1, - round: 0, - expectedError: ErrNodeIsScheduler, - }, // Timeout round needs to match current round. { signer: sk2, @@ -1337,7 +1338,7 @@ func TestExecutorTimeoutRequest(t *testing.T) { expectedError: nil, }, } { - err := pool.CheckProposerTimeout(ctx, childBlk, nl, tc.signer.Public(), tc.round) + err := pool.CheckSchedulerTimeout(ctx, childBlk, nl, tc.signer.Public(), tc.round) switch tc.expectedError { case nil: require.NoError(err, "CheckProposerTimeout unexpected error") @@ -1347,8 +1348,9 @@ func TestExecutorTimeoutRequest(t *testing.T) { } // Generate a commitment. - childBlk, _, ec := generateExecutorCommitment(t, pool.Round) + childBlk, _, ec := generateExecutorCommitment(t, pool.Round, committee) ec.NodeID = sk1.Public() + ec.Header.SchedulerID = sk1.Public() err := ec.Sign(sk1, rt.ID) require.NoError(err, "ec.Sign") // Adding commitment 1 should succeed. @@ -1356,7 +1358,7 @@ func TestExecutorTimeoutRequest(t *testing.T) { require.NoError(err, "AddExecutorCommitment") // Timeout after commitment should fail. - err = pool.CheckProposerTimeout(ctx, childBlk, nl, sk2.Public(), 0) + err = pool.CheckSchedulerTimeout(ctx, childBlk, nl, sk2.Public(), 0) require.Error(err, "CheckProposerTimeout commitment exists") require.Equal(ErrAlreadyCommitted, err, "CheckProposerTimeout commitment exists") }) @@ -1423,7 +1425,7 @@ func generateMockCommittee(t *testing.T, rtTemplate *registry.Runtime) ( return } -func generateExecutorCommitment(t *testing.T, round uint64) (*block.Block, *block.Block, ExecutorCommitment) { +func generateExecutorCommitment(t *testing.T, round uint64, committee *scheduler.Committee) (*block.Block, *block.Block, ExecutorCommitment) { var id common.Namespace childBlk := block.NewGenesisBlock(id, round) parentBlk := block.NewEmptyBlock(childBlk, 1, block.Normal) @@ -1446,6 +1448,14 @@ func generateExecutorCommitment(t *testing.T, round uint64) (*block.Block, *bloc }, } + // Use the first worker in the committee to be the scheduler. + for _, m := range committee.Members { + if m.Role == scheduler.RoleWorker { + ec.Header.SchedulerID = m.PublicKey + break + } + } + return childBlk, parentBlk, ec } @@ -1468,7 +1478,7 @@ func setupDiscrepancy( } // Generate a commitment. - childBlk, parentBlk, ec := generateExecutorCommitment(t, pool.Round) + childBlk, parentBlk, ec := generateExecutorCommitment(t, pool.Round, committee) ec1 := ec ec1.NodeID = sk1.Public() diff --git a/go/roothash/api/message/message_test.go b/go/roothash/api/message/message_test.go index 135861c2b3f..9e3c83e94bc 100644 --- a/go/roothash/api/message/message_test.go +++ b/go/roothash/api/message/message_test.go @@ -120,7 +120,7 @@ func newTestRuntime() *registry.Runtime { BatchFlushTimeout: 20 * time.Second, MaxBatchSize: 1, MaxBatchSizeBytes: 1024, - ProposerTimeout: 5, + SchedulerTimeout: 5, }, AdmissionPolicy: registry.RuntimeAdmissionPolicy{ EntityWhitelist: ®istry.EntityWhitelistRuntimeAdmissionPolicy{ diff --git a/go/roothash/tests/tester.go b/go/roothash/tests/tester.go index 02fd45d3e9a..d8dd304be4a 100644 --- a/go/roothash/tests/tester.go +++ b/go/roothash/tests/tester.go @@ -301,9 +301,6 @@ func (s *runtimeState) generateExecutorCommitments(t *testing.T, consensus conse _, ioRootHash, err := tree.Commit(ctx) require.NoError(err, "tree.Commit") - var emptyRoot hash.Hash - emptyRoot.Empty() - // Create the new block header that the nodes will commit to. parent = &block.Block{ Header: block.Header{ @@ -323,12 +320,16 @@ func (s *runtimeState) generateExecutorCommitments(t *testing.T, consensus conse msgsHash.Empty() inMsgsHash.Empty() + scheduler, err := executorCommittee.committee.Scheduler(child.Header.Round) + require.NoError(err, "committee.Scheduler") + // Generate all the executor commitments. executorNodes = append([]*registryTests.TestNode{}, executorCommittee.workers...) for _, node := range executorNodes { ec := commitment.ExecutorCommitment{ NodeID: node.Signer.Public(), Header: commitment.ExecutorCommitmentHeader{ + SchedulerID: scheduler.PublicKey, Header: commitment.ComputeResultsHeader{ Round: parent.Header.Round, PreviousHash: parent.Header.PreviousHash, @@ -442,8 +443,8 @@ func (s *runtimeState) testSuccessfulRound(t *testing.T, backend api.Backend, co finalizedProposals := make([]uint64, numNodes) missedProposals := make([]uint64, numNodes) - schedulerIdx, err := s.executorCommittee.committee.TransactionSchedulerIdx(header.Round - 1) - require.NoError(err, "TransactionSchedulerIdx") + schedulerIdx, err := s.executorCommittee.committee.SchedulerIdx(header.Round - 1) + require.NoError(err, "SchedulerIdx") finalizedProposals[schedulerIdx]++ // The first round has been finalized. require.EqualValues(finalizedProposals, state.LivenessStatistics.FinalizedProposals, "there should be one finalized proposal") @@ -543,12 +544,12 @@ WaitForRoundTimeoutBlocks: finalizedProposals := make([]uint64, numNodes) missedProposals := make([]uint64, numNodes) - schedulerIdx, err := s.executorCommittee.committee.TransactionSchedulerIdx(header.Round - 2) - require.NoError(err, "TransactionSchedulerIdx") + schedulerIdx, err := s.executorCommittee.committee.SchedulerIdx(header.Round - 2) + require.NoError(err, "SchedulerIdx") finalizedProposals[schedulerIdx]++ // The first round has been finalized. - schedulerIdx, err = s.executorCommittee.committee.TransactionSchedulerIdx(header.Round - 1) - require.NoError(err, "TransactionSchedulerIdx") + schedulerIdx, err = s.executorCommittee.committee.SchedulerIdx(header.Round - 1) + require.NoError(err, "SchedulerIdx") missedProposals[schedulerIdx]++ // The second round failed due to a timeout. require.EqualValues(finalizedProposals, state.LivenessStatistics.FinalizedProposals, "there should be one finalized proposal") @@ -681,7 +682,7 @@ WaitForProposerTimeoutBlocks: } // Wait for enough blocks so that proposer timeout is allowed. - if blk.Height >= startBlock+s.rt.Runtime.TxnScheduler.ProposerTimeout { + if blk.Height >= startBlock+s.rt.Runtime.TxnScheduler.SchedulerTimeout { break WaitForProposerTimeoutBlocks } case <-time.After(recvTimeout): @@ -691,8 +692,8 @@ WaitForProposerTimeoutBlocks: // Get scheduler at round. var scheduler *scheduler.CommitteeNode - scheduler, err = s.executorCommittee.committee.TransactionScheduler(child.Header.Round) - require.NoError(err, "roothash.TransactionScheduler") + scheduler, err = s.executorCommittee.committee.Scheduler(child.Header.Round) + require.NoError(err, "roothash.Scheduler") // Select node to trigger timeout. var timeoutNode *registryTests.TestNode @@ -744,16 +745,16 @@ WaitForProposerTimeoutBlocks: finalizedProposals := make([]uint64, numNodes) missedProposals := make([]uint64, numNodes) - schedulerIdx, err := s.executorCommittee.committee.TransactionSchedulerIdx(header.Round - 3) - require.NoError(err, "TransactionSchedulerIdx") + schedulerIdx, err := s.executorCommittee.committee.SchedulerIdx(header.Round - 3) + require.NoError(err, "SchedulerIdx") finalizedProposals[schedulerIdx]++ // The first round has been finalized. - schedulerIdx, err = s.executorCommittee.committee.TransactionSchedulerIdx(header.Round - 2) - require.NoError(err, "TransactionSchedulerIdx") + schedulerIdx, err = s.executorCommittee.committee.SchedulerIdx(header.Round - 2) + require.NoError(err, "SchedulerIdx") missedProposals[schedulerIdx]++ // The second round failed due to a timeout. - schedulerIdx, err = s.executorCommittee.committee.TransactionSchedulerIdx(header.Round - 1) - require.NoError(err, "TransactionSchedulerIdx") + schedulerIdx, err = s.executorCommittee.committee.SchedulerIdx(header.Round - 1) + require.NoError(err, "SchedulerIdx") missedProposals[schedulerIdx]++ // The third round failed due to a proposer timeout. require.EqualValues(finalizedProposals, state.LivenessStatistics.FinalizedProposals, "there should be one finalized proposal") diff --git a/go/runtime/txpool/txpool.go b/go/runtime/txpool/txpool.go index 71ead64cafc..48baed0b943 100644 --- a/go/runtime/txpool/txpool.go +++ b/go/runtime/txpool/txpool.go @@ -74,8 +74,11 @@ type TransactionPool interface { SubmitProposedBatch(batch [][]byte) // PromoteProposedBatch promotes the specified transactions that are already in the transaction - // pool into the current proposal queue. - PromoteProposedBatch(batch []hash.Hash) + // pool into the current proposal queue and returns a set of known transactions. + // + // For any missing transactions nil will be returned in their place and the map of missing + // transactions will be populated accordingly. + PromoteProposedBatch(batch []hash.Hash) ([]*TxQueueMeta, map[hash.Hash]int) // ClearProposedBatch clears the proposal queue. ClearProposedBatch() @@ -112,13 +115,6 @@ type TransactionPool interface { // ProcessIncomingMessages loads transactions from incoming messages into the pool. ProcessIncomingMessages(inMsgs []*message.IncomingMessage) error - // WakeupScheduler explicitly notifies subscribers that they should attempt scheduling. - WakeupScheduler() - - // WatchScheduler subscribes to notifications about when to attempt scheduling. The emitted - // boolean flag indicates whether the batch flush timeout expired. - WatchScheduler() (pubsub.ClosableSubscription, <-chan bool) - // WatchCheckedTransactions subscribes to notifications about new transactions being available // in the transaction pool for scheduling. WatchCheckedTransactions() (pubsub.ClosableSubscription, <-chan []*PendingCheckTransaction) @@ -175,9 +171,6 @@ type txPool struct { localQueue *localQueue mainQueue *mainQueue - schedulerTicker *time.Ticker - schedulerNotifier *pubsub.Broker - proposedTxsLock sync.Mutex proposedTxs map[hash.Hash]*TxQueueMeta @@ -193,7 +186,6 @@ func (t *txPool) Start() error { go t.checkWorker() go t.republishWorker() go t.recheckWorker() - go t.flushWorker() return nil } @@ -296,7 +288,7 @@ func (t *txPool) SubmitProposedBatch(batch [][]byte) { } } -func (t *txPool) PromoteProposedBatch(batch []hash.Hash) { +func (t *txPool) PromoteProposedBatch(batch []hash.Hash) ([]*TxQueueMeta, map[hash.Hash]int) { txs, missingTxs := t.GetKnownBatch(batch) if len(missingTxs) > 0 { t.logger.Debug("promoted proposed batch contains missing transactions", @@ -313,6 +305,8 @@ func (t *txPool) PromoteProposedBatch(batch []hash.Hash) { } t.proposedTxs[tx.Hash()] = tx } + + return txs, missingTxs } func (t *txPool) ClearProposedBatch() { @@ -389,11 +383,6 @@ func (t *txPool) ProcessBlock(bi *runtime.BlockInfo) error { close(t.initCh) fallthrough case bi.RuntimeBlock.Header.HeaderType == block.EpochTransition: - // Handle scheduler updates. - if err := t.updateScheduler(bi); err != nil { - return fmt.Errorf("failed to update scheduler: %w", err) - } - // Force recheck on epoch transitions. t.recheckTxCh.In() <- struct{}{} default: @@ -417,24 +406,6 @@ func (t *txPool) ProcessIncomingMessages(inMsgs []*message.IncomingMessage) erro return nil } -func (t *txPool) updateScheduler(bi *runtime.BlockInfo) error { - // Reset ticker to the new interval. - t.schedulerTicker.Reset(bi.ActiveDescriptor.TxnScheduler.BatchFlushTimeout) - - return nil -} - -func (t *txPool) WakeupScheduler() { - t.schedulerNotifier.Broadcast(false) -} - -func (t *txPool) WatchScheduler() (pubsub.ClosableSubscription, <-chan bool) { - sub := t.schedulerNotifier.Subscribe() - ch := make(chan bool) - sub.Unwrap(ch) - return sub, ch -} - func (t *txPool) WatchCheckedTransactions() (pubsub.ClosableSubscription, <-chan []*PendingCheckTransaction) { sub := t.checkTxNotifier.Subscribe() ch := make(chan []*PendingCheckTransaction) @@ -627,7 +598,6 @@ func (t *txPool) checkTxBatch(ctx context.Context, rr host.RichRuntime) { // Notify subscribers that we have received new transactions. t.checkTxNotifier.Broadcast(newTxs) - t.schedulerNotifier.Broadcast(false) } mainQueueSize.With(t.getMetricLabels()).Set(float64(t.mainQueue.inner.size())) @@ -865,22 +835,6 @@ func (t *txPool) recheck() { } } -func (t *txPool) flushWorker() { - // Wait for initialization to make sure that we have the scheduler available. - if err := t.ensureInitialized(); err != nil { - return - } - - for { - select { - case <-t.stopCh: - return - case <-t.schedulerTicker.C: - t.schedulerNotifier.Broadcast(true) - } - } -} - // New creates a new transaction pool instance. func New( runtimeID common.Namespace, @@ -925,8 +879,6 @@ func New( rimQueue: rq, localQueue: lq, mainQueue: mq, - schedulerTicker: time.NewTicker(1 * time.Hour), - schedulerNotifier: pubsub.NewBroker(false), proposedTxs: make(map[hash.Hash]*TxQueueMeta), republishCh: channels.NewRingChannel(1), }, nil diff --git a/go/scheduler/api/api.go b/go/scheduler/api/api.go index 0891b383561..2c302062498 100644 --- a/go/scheduler/api/api.go +++ b/go/scheduler/api/api.go @@ -4,6 +4,7 @@ package api import ( "context" "fmt" + "math" "strings" beacon "github.com/oasisprotocol/oasis-core/go/beacon/api" @@ -149,9 +150,9 @@ type Committee struct { ValidFor beacon.EpochTime `json:"valid_for"` } -// TransactionSchedulerIdx returns the index of the transaction scheduler +// SchedulerIdx returns the index of the primary transaction scheduler // within the committee for the provided round. -func (c *Committee) TransactionSchedulerIdx(round uint64) (int, error) { +func (c *Committee) SchedulerIdx(round uint64) (int, error) { var ( total uint64 worker uint64 @@ -164,11 +165,14 @@ func (c *Committee) TransactionSchedulerIdx(round uint64) (int, error) { total++ } + // The highest priority has the worker at position `round % total`. + primary := round % total + for idx, member := range c.Members { if member.Role != RoleWorker { continue } - if worker == round%total { + if worker == primary { return idx, nil } worker++ @@ -177,16 +181,48 @@ func (c *Committee) TransactionSchedulerIdx(round uint64) (int, error) { return 0, fmt.Errorf("no workers in committee") } -// TransactionScheduler returns the transaction scheduler of the committee +// Scheduler returns the transaction scheduler of the committee // based on the provided round. -func (c *Committee) TransactionScheduler(round uint64) (*CommitteeNode, error) { - idx, err := c.TransactionSchedulerIdx(round) +func (c *Committee) Scheduler(round uint64) (*CommitteeNode, error) { + idx, err := c.SchedulerIdx(round) if err != nil { return nil, err } return c.Members[idx], nil } +// SchedulingPriority returns the priority for a node to schedule transactions +// in the given round. Zero is the highest priority and infinity is the lowest. +// The latter is assigned to nodes that do not posses the worker role. +func (c *Committee) SchedulingPriority(round uint64, id signature.PublicKey) uint64 { + var ( + total uint64 + worker uint64 + isMember bool + ) + + for _, member := range c.Members { + if member.Role != RoleWorker { + continue + } + if member.PublicKey == id { + isMember = true + worker = total + } + total++ + } + + if !isMember { + return math.MaxUint64 + } + + // The highest priority has the worker at position `round % total`. + primary := round % total + priority := (total + worker - primary) % total + + return priority +} + // String returns a string representation of a Committee. func (c *Committee) String() string { members := make([]string, len(c.Members)) diff --git a/go/worker/client/committee/node.go b/go/worker/client/committee/node.go index 0646076297e..ca0f53e7c14 100644 --- a/go/worker/client/committee/node.go +++ b/go/worker/client/committee/node.go @@ -77,29 +77,40 @@ func (n *Node) Initialized() <-chan struct{} { return n.initCh } +// HandlePeerTx is guarded by CrossNode. func (n *Node) HandlePeerTx(ctx context.Context, tx []byte) error { // Nothing to do here. return nil } +// HandleNewConsensusBlockLocked is guarded by CrossNode. +func (n *Node) HandleNewConsensusBlockLocked(*consensus.Block) { + // Nothing to do here. +} + // HandleEpochTransitionLocked is guarded by CrossNode. func (n *Node) HandleEpochTransitionLocked(*committee.EpochSnapshot) { + // Nothing to do here. } // HandleNewBlockEarlyLocked is guarded by CrossNode. func (n *Node) HandleNewBlockEarlyLocked(*runtime.BlockInfo) { + // Nothing to do here. } // HandleNewBlockLocked is guarded by CrossNode. func (n *Node) HandleNewBlockLocked(blk *runtime.BlockInfo) { + // Nothing to do here. } // HandleNewEventLocked is guarded by CrossNode. func (n *Node) HandleNewEventLocked(*roothash.Event) { + // Nothing to do here. } // HandleRuntimeHostEventLocked is guarded by CrossNode. func (n *Node) HandleRuntimeHostEventLocked(*host.Event) { + // Nothing to do here. } func (n *Node) SubmitTx(ctx context.Context, tx []byte) (<-chan *api.SubmitTxResult, *protocol.Error, error) { diff --git a/go/worker/common/api/api.go b/go/worker/common/api/api.go index 8dc35301730..c0b70034768 100644 --- a/go/worker/common/api/api.go +++ b/go/worker/common/api/api.go @@ -117,8 +117,10 @@ type Status struct { // ExecutorRoles are the node's roles in the executor committee. ExecutorRoles []scheduler.Role `json:"executor_roles"` - // IsTransactionScheduler indicates whether the node is a transaction scheduler in this round. - IsTransactionScheduler bool `json:"is_txn_scheduler"` + // IsScheduler indicates whether the node is a transaction scheduler in this round. + IsScheduler bool `json:"is_txn_scheduler"` + // Priority is the node's priority to scheduler transactions in this round. + Priority uint64 `json:"priority"` // Liveness is the node's liveness status for the current epoch. Liveness *LivenessStatus `json:"liveness,omitempty"` diff --git a/go/worker/common/committee/group.go b/go/worker/common/committee/group.go index 3a9ef986406..faa73aa6415 100644 --- a/go/worker/common/committee/group.go +++ b/go/worker/common/committee/group.go @@ -3,6 +3,7 @@ package committee import ( "context" "fmt" + "math" "sync" beacon "github.com/oasisprotocol/oasis-core/go/beacon/api" @@ -37,8 +38,8 @@ type CommitteeInfo struct { // nolint: revive Indices []int Roles []scheduler.Role Committee *scheduler.Committee - PublicKeys map[signature.PublicKey]bool - Peers map[signature.PublicKey]bool + PublicKeys map[signature.PublicKey]struct{} + Peers map[signature.PublicKey]struct{} } // HasRole checks whether the node has the given role. @@ -134,19 +135,27 @@ func (e *EpochSnapshot) IsExecutorBackupWorker() bool { return e.executorCommittee.HasRole(scheduler.RoleBackupWorker) } -// IsTransactionScheduler checks if the current node is a a transaction scheduler -// at the specific runtime round. -func (e *EpochSnapshot) IsTransactionScheduler(round uint64) bool { +// IsScheduler checks if the current node is a transaction scheduler at the specific runtime round. +func (e *EpochSnapshot) IsScheduler(round uint64) bool { if e.executorCommittee == nil || e.executorCommittee.Committee == nil { return false } - scheduler, err := e.executorCommittee.Committee.TransactionScheduler(round) + scheduler, err := e.executorCommittee.Committee.Scheduler(round) if err != nil { return false } return scheduler.PublicKey.Equal(e.identity.NodeSigner.Public()) } +// SchedulingPriority returns the priority for a node to schedule transactions +// in the given round. +func (e *EpochSnapshot) SchedulingPriority(round uint64) uint64 { + if e.executorCommittee == nil || e.executorCommittee.Committee == nil { + return math.MaxUint64 + } + return e.executorCommittee.Committee.SchedulingPriority(round, e.identity.NodeSigner.Public()) +} + // Nodes returns a node descriptor lookup interface. func (e *EpochSnapshot) Nodes() nodes.NodeDescriptorLookup { return e.nodes @@ -163,22 +172,6 @@ func (e *EpochSnapshot) Node(ctx context.Context, id signature.PublicKey) (*node return n, nil } -// VerifyTxnSchedulerSigner verifies that the given signature comes from -// the transaction scheduler at provided round. -func (e *EpochSnapshot) VerifyTxnSchedulerSigner(id signature.PublicKey, round uint64) error { - if e.executorCommittee == nil || e.executorCommittee.Committee == nil { - return fmt.Errorf("epoch: no active transaction scheduler") - } - scheduler, err := e.executorCommittee.Committee.TransactionScheduler(round) - if err != nil { - return fmt.Errorf("epoch: error getting transaction scheduler: %w", err) - } - if !scheduler.PublicKey.Equal(id) { - return fmt.Errorf("epoch: signature is not from the transaction scheduler at round: %d", round) - } - return nil -} - // Group encapsulates communication with a group of nodes in the runtime committees. type Group struct { sync.RWMutex @@ -270,10 +263,10 @@ func (g *Group) EpochTransition(ctx context.Context, height int64) error { roles []scheduler.Role indices []int ) - publicKeys := make(map[signature.PublicKey]bool) - peers := make(map[signature.PublicKey]bool) + publicKeys := make(map[signature.PublicKey]struct{}) + peers := make(map[signature.PublicKey]struct{}) for index, member := range cm.Members { - publicKeys[member.PublicKey] = true + publicKeys[member.PublicKey] = struct{}{} if member.PublicKey.Equal(publicIdentity) { roles = append(roles, member.Role) indices = append(indices, index) @@ -285,7 +278,7 @@ func (g *Group) EpochTransition(ctx context.Context, height int64) error { return fmt.Errorf("group: failed to fetch node info: %w", err) } - peers[n.P2P.ID] = true + peers[n.P2P.ID] = struct{}{} } ci := &CommitteeInfo{ diff --git a/go/worker/common/committee/node.go b/go/worker/common/committee/node.go index dab7dc077ce..375ec52a4c0 100644 --- a/go/worker/common/committee/node.go +++ b/go/worker/common/committee/node.go @@ -137,6 +137,8 @@ type NodeHooks interface { // HandlePeerTx handles a transaction received from a (non-local) peer. HandlePeerTx(ctx context.Context, tx []byte) error + // Guarded by CrossNode. + HandleNewConsensusBlockLocked(blk *consensus.Block) // Guarded by CrossNode. HandleEpochTransitionLocked(*EpochSnapshot) // Guarded by CrossNode. @@ -327,7 +329,8 @@ func (n *Node) GetStatus(ctx context.Context) (*api.Status, error) { } } } - status.IsTransactionScheduler = epoch.IsTransactionScheduler(status.LatestRound) + status.IsScheduler = epoch.IsScheduler(status.LatestRound) + status.Priority = epoch.SchedulingPriority(status.LatestRound) status.Peers = n.P2P.Peers(n.Runtime.ID()) @@ -342,6 +345,11 @@ func (n *Node) getMetricLabels() prometheus.Labels { } } +// HandleNewConsensusBlockLocked is guarded by CrossNode. +func (n *Node) HandleNewConsensusBlockLocked(*consensus.Block) { + // Nothing to do here. +} + // Guarded by n.CrossNode. func (n *Node) handleEpochTransitionLocked(height int64) { n.logger.Info("epoch transition has occurred") @@ -456,6 +464,15 @@ func (n *Node) updateHostedRuntimeVersionLocked() { } } +// Guarded by n.CrossNode. +func (n *Node) handleNewConsensusBlockLocked(blk *consensus.Block) { + n.Height = blk.Height + + for _, hooks := range n.hooks { + hooks.HandleNewConsensusBlockLocked(blk) + } +} + // Guarded by n.CrossNode. func (n *Node) handleNewBlockLocked(blk *block.Block, height int64) { processedBlockCount.With(n.getMetricLabels()).Inc() @@ -759,7 +776,7 @@ func (n *Node) worker() { func() { n.CrossNode.Lock() defer n.CrossNode.Unlock() - n.Height = blk.Height + n.handleNewConsensusBlockLocked(blk) }() case blk := <-blocks: // We are initialized after we have received the first block. This makes sure that any diff --git a/go/worker/compute/executor/committee/batch.go b/go/worker/compute/executor/committee/batch.go deleted file mode 100644 index 239984d00a4..00000000000 --- a/go/worker/compute/executor/committee/batch.go +++ /dev/null @@ -1,71 +0,0 @@ -package committee - -import ( - "fmt" - - "github.com/oasisprotocol/oasis-core/go/common/crypto/hash" - "github.com/oasisprotocol/oasis-core/go/roothash/api/commitment" - "github.com/oasisprotocol/oasis-core/go/runtime/transaction" - "github.com/oasisprotocol/oasis-core/go/runtime/txpool" -) - -// unresolvedBatch is a batch that may still need to be resolved (fetched from storage). -type unresolvedBatch struct { - proposal *commitment.Proposal - - batch transaction.RawBatch - missingTxs map[hash.Hash]int - - maxBatchSizeBytes uint64 -} - -func (ub *unresolvedBatch) String() string { - switch { - case ub.proposal != nil: - return fmt.Sprintf("UnresolvedBatch{hash: %s}", ub.proposal.Header.BatchHash) - default: - return "UnresolvedBatch{?}" - } -} - -func (ub *unresolvedBatch) hash() hash.Hash { - if ub.proposal == nil { - return hash.Hash{} - } - return ub.proposal.Header.BatchHash -} - -func (ub *unresolvedBatch) resolve(txPool txpool.TransactionPool) (transaction.RawBatch, error) { - if ub.batch != nil { - return ub.batch, nil - } - if ub.proposal == nil { - return nil, fmt.Errorf("resolve called on unresolvable batch") - } - if len(ub.proposal.Batch) == 0 { - return transaction.RawBatch{}, nil - } - - resolvedBatch, missingTxs := txPool.GetKnownBatch(ub.proposal.Batch) - if len(missingTxs) > 0 { - ub.missingTxs = missingTxs - return nil, nil - } - ub.missingTxs = nil - - var ( - batch transaction.RawBatch - totalSizeBytes int - ) - for _, checkedTx := range resolvedBatch { - totalSizeBytes = totalSizeBytes + checkedTx.Size() - if ub.maxBatchSizeBytes > 0 && uint64(totalSizeBytes) > ub.maxBatchSizeBytes { - return nil, fmt.Errorf("batch too large (max: %d size: >=%d)", ub.maxBatchSizeBytes, totalSizeBytes) - } - - batch = append(batch, checkedTx.Raw()) - } - ub.batch = batch - - return batch, nil -} diff --git a/go/worker/compute/executor/committee/discrepancy.go b/go/worker/compute/executor/committee/discrepancy.go index 942db7ffd24..bdf9abbc960 100644 --- a/go/worker/compute/executor/committee/discrepancy.go +++ b/go/worker/compute/executor/committee/discrepancy.go @@ -1,57 +1,47 @@ package committee import ( + "context" + "github.com/oasisprotocol/oasis-core/go/common/crash" roothash "github.com/oasisprotocol/oasis-core/go/roothash/api" "github.com/oasisprotocol/oasis-core/go/roothash/api/commitment" ) -func (n *Node) handleDiscrepancyLocked(height uint64) { - n.logger.Warn("execution discrepancy detected") +type discrepancyEvent struct { + priority uint64 + height uint64 +} + +func (n *Node) NotifyDiscrepancy(info *discrepancyEvent) { + // Drop discrepancies if the worker falls behind. + select { + case <-n.discrepancyCh: + default: + } + + // Non-blocking send. + n.discrepancyCh <- info +} + +func (n *Node) handleDiscrepancy(ctx context.Context, info *discrepancyEvent) { + n.logger.Warn("execution discrepancy detected", + "priority", info.priority, + "height", info.height, + ) crash.Here(crashPointDiscrepancyDetectedAfter) discrepancyDetectedCount.With(n.getMetricLabels()).Inc() - // If the node is not a backup worker in this epoch, no need to do anything. Also if the - // node is an executor worker in this epoch, then it has already processed and submitted - // a commitment, so no need to do anything. - epoch := n.commonNode.Group.GetEpochSnapshot() - if !epoch.IsExecutorBackupWorker() || epoch.IsExecutorWorker() { - return - } - // Make sure that the runtime has synced this consensus block. - if rt := n.commonNode.GetHostedRuntime(); rt != nil { - err := rt.ConsensusSync(n.roundCtx, height) - if err != nil { - n.logger.Warn("failed to ask the runtime to sync the latest consensus block", - "err", err, - "height", height, - ) - } - } - - var state StateWaitingForEvent - switch s := n.state.(type) { - case StateWaitingForBatch: - // Discrepancy detected event received before the batch. We need to remember that there was - // a discrepancy and keep waiting for the batch. - s.discrepancyDetected = true - n.transitionLocked(s) - return - case StateWaitingForEvent: - state = s - default: - n.logger.Warn("ignoring received discrepancy event in incorrect state", - "state", s, + err := n.rt.ConsensusSync(ctx, info.height) + if err != nil { + n.logger.Warn("failed to ask the runtime to sync the latest consensus block", + "err", err, + "height", info.height, ) - return } - - // Backup worker, start processing a batch. - n.logger.Info("backup worker activating and processing batch") - n.startProcessingBatchLocked(state.batch) } // HandleNewEventLocked implements NodeHooks. @@ -59,26 +49,20 @@ func (n *Node) handleDiscrepancyLocked(height uint64) { func (n *Node) HandleNewEventLocked(ev *roothash.Event) { switch { case ev.ExecutionDiscrepancyDetected != nil: - n.handleDiscrepancyLocked(uint64(ev.Height)) + n.NotifyDiscrepancy(&discrepancyEvent{ + priority: ev.ExecutionDiscrepancyDetected.Priority, + height: uint64(ev.Height), + }) } } -func (n *Node) handleObservedExecutorCommitment(ec *commitment.ExecutorCommitment) { - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() - - // Don't do anything if we are not a backup worker or we are an executor worker. - es := n.commonNode.Group.GetEpochSnapshot() - if !es.IsExecutorBackupWorker() || es.IsExecutorWorker() { - return - } - +func (n *Node) handleObservedExecutorCommitment(ctx context.Context, ec *commitment.ExecutorCommitment) { n.logger.Debug("observed executor commitment", "commitment", ec, ) // Make sure the executor commitment is for the next round. - currentRound := n.commonNode.CurrentBlock.Header.Round + currentRound := n.blockInfo.RuntimeBlock.Header.Round nextRound := currentRound + 1 if ec.Header.Header.Round != nextRound { n.logger.Debug("observed executor commitment is not for the next round", @@ -90,14 +74,14 @@ func (n *Node) handleObservedExecutorCommitment(ec *commitment.ExecutorCommitmen // Initialize the pool if needed. if n.commitPool.Round != currentRound { - n.commitPool.Runtime = es.GetRuntime() - n.commitPool.Committee = es.GetExecutorCommittee().Committee + n.commitPool.Runtime = n.epoch.GetRuntime() + n.commitPool.Committee = n.epoch.GetExecutorCommittee().Committee n.commitPool.ResetCommitments(currentRound) } // TODO: Handle equivocation detection. - err := n.commitPool.AddExecutorCommitment(n.ctx, n.commonNode.CurrentBlock, es, ec, nil) + err := n.commitPool.AddExecutorCommitment(ctx, n.blockInfo.RuntimeBlock, n.epoch, ec, nil) if err != nil { n.logger.Debug("ignoring bad observed executor commitment", "err", err, @@ -107,9 +91,17 @@ func (n *Node) handleObservedExecutorCommitment(ec *commitment.ExecutorCommitmen } // In case observed commits indicate a discrepancy, preempt consensus and immediately handle. - if _, err = n.commitPool.ProcessCommitments(false); err == commitment.ErrDiscrepancyDetected { - n.logger.Warn("observed commitments indicate discrepancy") - - n.handleDiscrepancyLocked(uint64(n.commonNode.CurrentBlockHeight)) + if _, err = n.commitPool.ProcessCommitments(false); err != commitment.ErrDiscrepancyDetected { + return } + + schedulerID, _ := n.commitPool.Scheduler() + priority := n.commitPool.Committee.SchedulingPriority(n.commitPool.Round, schedulerID) + + n.logger.Warn("observed commitments indicate discrepancy") + + n.NotifyDiscrepancy(&discrepancyEvent{ + priority: priority, + height: uint64(n.blockInfo.ConsensusBlock.Height), + }) } diff --git a/go/worker/compute/executor/committee/hooks.go b/go/worker/compute/executor/committee/hooks.go new file mode 100644 index 00000000000..c95b5cdb126 --- /dev/null +++ b/go/worker/compute/executor/committee/hooks.go @@ -0,0 +1,60 @@ +package committee + +import ( + "context" + + "github.com/oasisprotocol/oasis-core/go/common/crash" + consensus "github.com/oasisprotocol/oasis-core/go/consensus/api" + runtime "github.com/oasisprotocol/oasis-core/go/runtime/api" + "github.com/oasisprotocol/oasis-core/go/worker/common/committee" +) + +// Ensure Node implements NodeHooks. +var _ committee.NodeHooks = (*Node)(nil) + +// HandlePeerTx implements NodeHooks. +func (n *Node) HandlePeerTx(ctx context.Context, tx []byte) error { + // Nothing to do here. + return nil +} + +// HandleNewConsensusBlockLocked implements NodeHooks. +// Guarded by n.commonNode.CrossNode. +func (n *Node) HandleNewConsensusBlockLocked(blk *consensus.Block) { + // Drop blocks if the worker falls behind. + select { + case <-n.consensusBlockCh: + default: + } + + // Non-blocking send. + n.consensusBlockCh <- blk +} + +// HandleEpochTransitionLocked implements NodeHooks. +// Guarded by n.commonNode.CrossNode. +func (n *Node) HandleEpochTransitionLocked(epoch *committee.EpochSnapshot) { + // Nothing to do here. +} + +// HandleNewBlockEarlyLocked implements NodeHooks. +// Guarded by n.commonNode.CrossNode. +func (n *Node) HandleNewBlockEarlyLocked(bi *runtime.BlockInfo) { + crash.Here(crashPointRoothashReceiveAfter) + + // Update our availability. + n.nudgeAvailabilityLocked(false) +} + +// HandleNewBlockLocked implements NodeHooks. +// Guarded by n.commonNode.CrossNode. +func (n *Node) HandleNewBlockLocked(bi *runtime.BlockInfo) { + // Drop blocks if the worker falls behind. + select { + case <-n.blockInfoCh: + default: + } + + // Non-blocking send. + n.blockInfoCh <- bi +} diff --git a/go/worker/compute/executor/committee/node.go b/go/worker/compute/executor/committee/node.go index 6b27ecb9fae..30a17096460 100644 --- a/go/worker/compute/executor/committee/node.go +++ b/go/worker/compute/executor/committee/node.go @@ -4,9 +4,12 @@ import ( "context" "errors" "fmt" + "math" "sync" "time" + "golang.org/x/exp/maps" + beacon "github.com/oasisprotocol/oasis-core/go/beacon/api" "github.com/oasisprotocol/oasis-core/go/common/crash" "github.com/oasisprotocol/oasis-core/go/common/crypto/hash" @@ -16,7 +19,6 @@ import ( "github.com/oasisprotocol/oasis-core/go/common/version" consensus "github.com/oasisprotocol/oasis-core/go/consensus/api" p2p "github.com/oasisprotocol/oasis-core/go/p2p/api" - p2pError "github.com/oasisprotocol/oasis-core/go/p2p/error" p2pProtocol "github.com/oasisprotocol/oasis-core/go/p2p/protocol" roothash "github.com/oasisprotocol/oasis-core/go/roothash/api" "github.com/oasisprotocol/oasis-core/go/roothash/api/block" @@ -34,13 +36,13 @@ import ( ) var ( - errSeenNewerBlock = fmt.Errorf("executor: seen newer block") - errRuntimeAborted = fmt.Errorf("executor: runtime aborted batch processing") - errBatchTooLarge = p2pError.Permanent(fmt.Errorf("executor: batch too large")) errMsgFromNonTxnSched = fmt.Errorf("executor: received txn scheduler dispatch msg from non-txn scheduler") - // proposeTimeoutDelay is the duration to wait before submitting the propose timeout request. - proposeTimeoutDelay = 2 * time.Second + // priorityDelay is the duration to wait before accepting proposal + // from the next backup scheduler. + priorityDelay = 2 * time.Second + // timeoutDelay is the duration to wait before submitting the propose timeout request. + timeoutDelay = 2 * time.Second // abortTimeout is the duration to wait for the runtime to abort. abortTimeout = 5 * time.Second // getInfoTimeout is the maximum time the runtime can spend replying to GetInfo. @@ -54,11 +56,6 @@ type Node struct { // nolint: maligned runtimeTrustSynced bool runtimeTrustSyncCncl context.CancelFunc - // Guarded by .commonNode.CrossNode. - proposingTimeout bool - missingTxsCancel context.CancelFunc - pendingProposals *pendingProposals - commonNode *committee.Node commonCfg commonWorker.Config roleProvider registration.RoleProvider @@ -72,22 +69,37 @@ type Node struct { // nolint: maligned quitCh chan struct{} initCh chan struct{} - // Mutable and shared with common node's worker. - // Guarded by .commonNode.CrossNode. - state NodeState - // Context valid until the next round. - // Guarded by .commonNode.CrossNode. - roundCtx context.Context - roundCancelCtx context.CancelFunc - - commitPool commitment.Pool - storage storage.LocalBackend txSync txsync.Client + // Global, used by every round worker. + + state NodeState stateTransitions *pubsub.Broker - // Bump this when we need to change what the worker selects over. - reselect chan struct{} + proposals *proposalQueue + commitPool *commitment.Pool + + blockInfoCh chan *runtime.BlockInfo + processedBatchCh chan *processedBatch + consensusBlockCh chan *consensus.Block + discrepancyCh chan *discrepancyEvent + reselectCh chan struct{} + + txCh <-chan []*txpool.PendingCheckTransaction + ecCh <-chan *commitment.ExecutorCommitment + + // Local, set and used by every round worker. + + rt host.RichRuntime + epoch *committee.EpochSnapshot + blockInfo *runtime.BlockInfo + rtState *roothash.RuntimeState + roundResults *roothash.RoundResults + submitted map[uint64]struct{} + priority uint64 + timeoutActive bool + timeoutTimer *time.Timer + proposedBatch *proposedBatch logger *logging.Logger } @@ -143,46 +155,15 @@ func (n *Node) WatchStateTransitions() (<-chan NodeState, *pubsub.Subscription) return ch, sub } -func (n *Node) processProposal(ctx context.Context, proposal *commitment.Proposal) error { - rt, err := n.commonNode.Runtime.ActiveDescriptor(ctx) - if err != nil { - n.logger.Warn("failed to fetch active runtime descriptor", - "err", err, - ) - // Do not forward the proposal further as we are unable to validate it. - return p2pError.Permanent(err) - } - - // Do a quick check on the batch size. - if uint64(len(proposal.Batch)) > rt.TxnScheduler.MaxBatchSize { - n.logger.Warn("received proposed batch contained too many transactions", - "max_batch_size", rt.TxnScheduler.MaxBatchSize, - "batch_size", len(proposal.Batch), - ) - // Do not forward the proposal further as it seems invalid. - return errBatchTooLarge - } - - batch := &unresolvedBatch{ - proposal: proposal, - maxBatchSizeBytes: rt.TxnScheduler.MaxBatchSizeBytes, - } - - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() - return n.handleProposalLocked(batch) -} - -func (n *Node) bumpReselect() { +func (n *Node) reselect() { select { - case n.reselect <- struct{}{}: + case n.reselectCh <- struct{}{}: default: // If there's one already queued, we don't need to do anything. } } -// Guarded by n.commonNode.CrossNode. -func (n *Node) transitionLocked(state NodeState) { +func (n *Node) transitionState(state NodeState) { n.logger.Info("state transition", "current_state", n.state, "new_state", state, @@ -205,209 +186,214 @@ func (n *Node) transitionLocked(state NodeState) { n.state = state n.stateTransitions.Broadcast(state) - // Restart our worker's select in case our state-specific channels have changed. - n.bumpReselect() } -// HandleEpochTransitionLocked implements NodeHooks. -// Guarded by n.commonNode.CrossNode. -func (n *Node) HandleEpochTransitionLocked(epoch *committee.EpochSnapshot) { - switch { - case epoch.IsExecutorWorker(), epoch.IsExecutorBackupWorker(): - n.transitionLocked(StateWaitingForBatch{}) - default: - n.transitionLocked(StateNotReady{}) - } -} +func (n *Node) transitionStateToProcessing(ctx context.Context, proposal *commitment.Proposal, priority uint64, batch transaction.RawBatch) { + ctx, cancel := context.WithCancel(ctx) + done := make(chan struct{}) -// HandleNewBlockEarlyLocked implements NodeHooks. -// Guarded by n.commonNode.CrossNode. -func (n *Node) HandleNewBlockEarlyLocked(bi *runtime.BlockInfo) { - crash.Here(crashPointRoothashReceiveAfter) + n.transitionState(StateProcessingBatch{ + mode: protocol.ExecutionModeExecute, + priority: priority, + batchStartTime: time.Now(), + cancelFn: cancel, + done: done, + }) - // If we have seen a new block while a batch was processing, we need to - // abort it no matter what as any processed state may be invalid. - n.abortBatchLocked(errSeenNewerBlock) - // Update our availability. - n.nudgeAvailabilityLocked(false) + // Request the worker host to process a batch. This is done in a separate + // goroutine so that the runtime worker can continue processing events. + go func() { + defer close(done) + n.startProcessingBatch(ctx, proposal, priority, batch) + }() } -// HandleNewBlockLocked implements NodeHooks. -// Guarded by n.commonNode.CrossNode. -func (n *Node) HandleNewBlockLocked(bi *runtime.BlockInfo) { - header := bi.RuntimeBlock.Header +func (n *Node) transitionStateToProcessingFailure( + ctx context.Context, + proposal *commitment.Proposal, + priority uint64, + bytes uint64, + maxBytes uint64, + batchSize uint64, + maxBatchSize uint64, +) { + n.logger.Debug("batch too large", + "bytes", bytes, + "max_bytes", maxBytes, + "batch_size", batchSize, + "max_batch_size", maxBatchSize, + ) - // Cancel old round context, start a new one. - if n.roundCancelCtx != nil { - (n.roundCancelCtx)() - } - n.roundCtx, n.roundCancelCtx = context.WithCancel(n.ctx) + cancel := func() {} + done := make(chan struct{}) + close(done) - clearProposalQueue := true + n.transitionState(StateProcessingBatch{ + mode: protocol.ExecutionModeExecute, + priority: priority, + batchStartTime: time.Now(), + cancelFn: cancel, + done: done, + }) - // Perform actions based on current state. + // Submit response to the round worker. + n.processedBatchCh <- &processedBatch{ + proposal: proposal, + priority: priority, + computed: nil, + raw: nil, + } +} + +func (n *Node) updateState(ctx context.Context, minPriority uint64, maxPriority uint64, discrepancy bool) { switch state := n.state.(type) { + case StateWaitingForBatch: + // Nothing to be done here. case StateWaitingForTxs: - // Stop waiting for transactions and start a new round. - n.logger.Warn("considering previous proposal invalid due to missing transactions") - n.transitionLocked(StateWaitingForBatch{}) - case StateWaitingForEvent: - // Block finalized without the need for a backup worker. - n.logger.Info("considering the round finalized", - "round", header.Round, - "header_hash", header.EncodedHash(), - ) - n.transitionLocked(StateWaitingForBatch{}) - case StateWaitingForFinalize: - func() { - defer n.transitionLocked(StateWaitingForBatch{}) - - // A new block means the round has been finalized. - n.logger.Info("considering the round finalized", - "round", header.Round, - "header_hash", header.EncodedHash(), - "header_type", header.HeaderType, - ) - if header.HeaderType != block.Normal { - return - } - if !header.IORoot.Equal(&state.proposedIORoot) { - n.logger.Error("proposed batch was not finalized", - "header_io_root", header.IORoot, - "proposed_io_root", state.proposedIORoot, - "header_type", header.HeaderType, - "batch_size", len(state.txHashes), - ) + switch { + case state.priority < minPriority || state.priority > maxPriority: + // Priority ouf ot bounds; stop fetching. + state.Cancel() + n.transitionState(StateWaitingForBatch{}) + case state.maxBytes > 0 && state.bytes > state.maxBytes: + // Some transactions have been received, but the batch is too large; stop fetching and + // submit failure. + state.Cancel() + + // All workers should indicate failure immediately. + n.transitionStateToProcessingFailure(ctx, state.proposal, state.priority, state.bytes, state.maxBytes, state.batchSize, state.maxBatchSize) + case len(state.txs) == 0: + // All transactions have been received; stop fetching and start processing. + state.Cancel() + + // The backup workers should process only if the discrepancy was detected. + if !n.epoch.IsExecutorWorker() && n.epoch.IsExecutorBackupWorker() && !discrepancy { + n.transitionState(StateWaitingForEvent{ + proposal: state.proposal, + priority: state.priority, + batch: state.batch, + }) return } - // Record time taken for successfully processing a batch. - batchProcessingTime.With(n.getMetricLabels()).Observe(time.Since(state.batchStartTime).Seconds()) - - n.logger.Debug("removing processed batch from queue", - "batch_size", len(state.txHashes), - "io_root", header.IORoot, - ) - // Remove processed transactions from queue. - n.commonNode.TxPool.HandleTxsUsed(state.txHashes) - }() - } - - // Clear proposal queue. - if clearProposalQueue { - n.commonNode.TxPool.ClearProposedBatch() + n.transitionStateToProcessing(ctx, state.proposal, state.priority, state.batch) + default: + // Keep on waiting for transactions. + } + case StateWaitingForEvent: + if state.priority < minPriority || state.priority > maxPriority { + // Priority ouf ot bounds; stop fetching. + n.transitionState(StateWaitingForBatch{}) + return + } + if discrepancy { + // Discrepancy detected; stop waiting and start processing. + n.transitionStateToProcessing(ctx, state.proposal, state.priority, state.batch) + return + } + case StateProcessingBatch: + if state.priority < minPriority || state.priority > maxPriority { + // Priority ouf ot bounds; stop processing. + n.abortBatch(&state) + n.transitionState(StateWaitingForBatch{}) + return + } } +} - // Clear the potentially set "is proposing timeout" flag from the previous round. - n.proposingTimeout = false - - // Check if we are a proposer and if so try to immediately schedule a new batch. - if n.commonNode.Group.GetEpochSnapshot().IsTransactionScheduler(header.Round) { - n.logger.Info("we are a transaction scheduler", - "round", header.Round, - ) - - n.commonNode.TxPool.WakeupScheduler() - } +func (n *Node) handleNewConsensusBlock(ctx context.Context, blk *consensus.Block) { + n.logger.Debug("handling new consensus block", + "height", blk.Height, + ) - // Check if we have any pending proposals and attempt to handle them. - n.handlePendingProposalsLocked() + n.scheduleTimeout(ctx, blk) } -func (n *Node) proposeTimeoutLocked(roundCtx context.Context) error { +func (n *Node) scheduleTimeout(ctx context.Context, blk *consensus.Block) { // Do not propose a timeout if we are already proposing it. // The flag will get cleared on the next round or if the propose timeout // tx fails. - if n.proposingTimeout { - return nil + if n.timeoutActive { + return } - if n.commonNode.CurrentBlock == nil { - return fmt.Errorf("executor: propose timeout error, nil block") + timeoutHeight := n.blockInfo.ConsensusBlock.Height + n.blockInfo.ActiveDescriptor.TxnScheduler.SchedulerTimeout + + n.logger.Debug("considering to propose a timeout", + "height", blk.Height, + "runtime_block_height", n.blockInfo.ConsensusBlock.Height, + "timeout_height", timeoutHeight, + ) + + if blk.Height < timeoutHeight { + n.logger.Debug("timeout not reached yet") + return } - rt, err := n.commonNode.Runtime.ActiveDescriptor(roundCtx) - if err != nil { - return err + n.logger.Debug("requesting timeout") + + // Wait a bit before actually proposing a timeout, to give the current + // scheduler some time to propose a batch in case it just received it. + // + // This prevents triggering a timeout when there is a long period + // of no transactions, as without this artificial delay, the non + // scheduler nodes would be faster in proposing a timeout than the + // scheduler node proposing a batch. + _ = n.timeoutTimer.Stop() + select { + case <-n.timeoutTimer.C: + default: } - proposerTimeout := rt.TxnScheduler.ProposerTimeout - currentBlockHeight := n.commonNode.CurrentBlockHeight - if n.commonNode.Height < currentBlockHeight+proposerTimeout { - n.logger.Debug("executor: proposer timeout not reached yet", - "height", n.commonNode.Height, - "current_block_height", currentBlockHeight, - "proposer_timeout", proposerTimeout, + n.timeoutTimer.Reset(timeoutDelay) + n.timeoutActive = true +} + +func (n *Node) handleTimeout(ctx context.Context, priority uint64) { + round := n.blockInfo.RuntimeBlock.Header.Round + height := n.blockInfo.ConsensusBlock.Height + + // Propose timeout only if we are not allowed to propose. + if n.priority <= priority { + n.logger.Debug("not requesting timeout, waiting to propose a batch", + "round", round, + "height", height, ) - return nil + return } - n.logger.Debug("executor requesting proposer timeout", - "height", n.commonNode.Height, - "current_block_height", currentBlockHeight, - "proposer_timeout", proposerTimeout, - ) - n.proposingTimeout = true - tx := roothash.NewRequestProposerTimeoutTx(0, nil, n.commonNode.Runtime.ID(), n.commonNode.CurrentBlock.Header.Round) - go func(round uint64) { - // Wait a bit before actually proposing a timeout, to give the current - // scheduler some time to propose a batch in case it just received it. - // - // This prevents triggering a timeout when there is a long period - // of no transactions, as without this artificial delay, the non - // scheduler nodes would be faster in proposing a timeout than the - // scheduler node proposing a batch. - select { - case <-time.After(proposeTimeoutDelay): - case <-roundCtx.Done(): - n.logger.Info("not requesting proposer timeout, round context canceled") - return - } + // Propose timeout only if no proposals have been received so far via P2P network. + if _, _, ok := n.proposals.Best(round+1, 0, math.MaxUint64); ok { + n.logger.Debug("not requesting timeout, at least one proposal received", + "round", round, + "height", height, + ) + return + } - // Make sure we are still in the right state/round. - n.commonNode.CrossNode.Lock() - // Make sure we are still in the right state. - var invalidState bool - switch n.state.(type) { - case StateWaitingForBatch, StateWaitingForTxs: - default: - invalidState = true - } - // Make sure we are still processing the right round. - if round != n.commonNode.CurrentBlock.Header.Round { - invalidState = true - } - if invalidState { - n.logger.Info("not requesting proposer timeout", - "height", n.commonNode.Height, - "current_block_round", n.commonNode.CurrentBlock.Header.Round, - "proposing_round", round, - "state", n.state, - ) - n.commonNode.CrossNode.Unlock() - return - } - n.commonNode.CrossNode.Unlock() + // Propose timeout only if no commitments have been observed so far via consensus layer. + if n.commitPool.Round == round && len(n.commitPool.ExecuteCommitments) > 0 { + n.logger.Debug("not requesting timeout, at least one commitment observed", + "round", round, + "height", height, + ) + return + } - err := consensus.SignAndSubmitTx(roundCtx, n.commonNode.Consensus, n.commonNode.Identity.NodeSigner, tx) - switch err { - case nil: - n.logger.Info("executor timeout request finalized", - "height", n.commonNode.Height, - "current_block_height", currentBlockHeight, - "proposer_timeout", proposerTimeout, - ) - default: - n.logger.Error("failed to submit executor timeout request", - "height", n.commonNode.Height, - "current_block_height", currentBlockHeight, - "err", err, - ) - n.commonNode.CrossNode.Lock() - n.proposingTimeout = false - n.commonNode.CrossNode.Unlock() - } - }(n.commonNode.CurrentBlock.Header.Round) + tx := roothash.NewRequestProposerTimeoutTx(0, nil, n.commonNode.Runtime.ID(), round) + err := consensus.SignAndSubmitTx(ctx, n.commonNode.Consensus, n.commonNode.Identity.NodeSigner, tx) + if err != nil { + n.logger.Error("failed to submit timeout request", + "round", round, + "height", height, + "err", err, + ) + n.timeoutActive = false + return + } - return nil + n.logger.Info("timeout request finalized", + "round", round, + "height", height, + ) } func (n *Node) getRtStateAndRoundResults(ctx context.Context, height int64) (*roothash.RuntimeState, *roothash.RoundResults, error) { @@ -435,58 +421,49 @@ func (n *Node) getRtStateAndRoundResults(ctx context.Context, height int64) (*ro return state, roundResults, nil } -func (n *Node) handleScheduleBatch(force bool) { // nolint: gocyclo - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() +func (n *Node) scheduleBatch(ctx context.Context, round uint64, force bool) { + n.logger.Debug("trying to schedule a batch", + "round", round, + "priority", n.priority, + "force", force, + "state", n.state.Name(), + ) // Check if we are in a suitable state for scheduling a batch. switch n.state.(type) { case StateWaitingForBatch: - // We are waiting for a batch. - case StateWaitingForTxs: - // We are waiting for transactions. Note that this means we are not a transaction - // scheduler and so we won't actually be able to schedule anything. But we should still - // propose a timeout if the transaction scheduler proposed something that nobody has. default: - n.logger.Debug("not scheduling a batch, incorrect state", "state", n.state) return } - if n.commonNode.CurrentBlock == nil { - n.logger.Debug("not scheduling a batch, no blocks") + + // Schedule only once. + if _, ok := n.submitted[n.priority]; ok { + n.logger.Debug("not scheduling, commitment already submitted") return } - epoch := n.commonNode.Group.GetEpochSnapshot() - // If we are not an executor worker in this epoch, we don't need to do anything. - if !epoch.IsExecutorWorker() { - n.logger.Debug("not scheduling a batch, not an executor") + // Only executor workers are permitted to schedule batches. + if !n.epoch.IsExecutorWorker() { + n.logger.Debug("not scheduling, not an executor") return } // If the next block will be an epoch transition block, do not propose anything as it will be // reverted anyway (since the committee will change). - epochState, err := n.commonNode.Consensus.Beacon().GetFutureEpoch(n.roundCtx, n.commonNode.CurrentBlockHeight) + epochState, err := n.commonNode.Consensus.Beacon().GetFutureEpoch(ctx, n.blockInfo.ConsensusBlock.Height) // TODO: is this height ok? if err != nil { n.logger.Error("failed to fetch future epoch state", "err", err, ) return } - if epochState != nil && epochState.Height == n.commonNode.CurrentBlockHeight+1 { - n.logger.Debug("not scheduling a batch, next consensus block is an epoch transition") - return - } - - rtState, roundResults, err := n.getRtStateAndRoundResults(n.roundCtx, n.commonNode.CurrentBlockHeight) - if err != nil { - n.logger.Debug("not scheduling a batch", - "err", err, - ) + if epochState != nil && epochState.Height == n.blockInfo.ConsensusBlock.Height+1 { // TODO: is this height ok? + n.logger.Debug("not scheduling, next consensus block is an epoch transition") return } // Fetch incoming message queue metadata to see if there's any queued messages. - inMsgMeta, err := n.commonNode.Consensus.RootHash().GetIncomingMessageQueueMeta(n.roundCtx, &roothash.RuntimeRequest{ + inMsgMeta, err := n.commonNode.Consensus.RootHash().GetIncomingMessageQueueMeta(ctx, &roothash.RuntimeRequest{ RuntimeID: n.commonNode.Runtime.ID(), // We make the check at the latest height even though we will later only look at the last // height. This will make sure that any messages eventually get processed even if there are @@ -502,14 +479,9 @@ func (n *Node) handleScheduleBatch(force bool) { // nolint: gocyclo } // Check what the runtime supports. - rt := n.commonNode.GetHostedRuntime() - if rt == nil { - n.logger.Debug("not scheduling a batch, the runtime is not yet ready") - return - } - rtInfo, err := rt.GetInfo(n.roundCtx) + rtInfo, err := n.rt.GetInfo(ctx) if err != nil { - n.logger.Warn("not scheduling a batch, the runtime is broken", + n.logger.Warn("not scheduling, the runtime is broken", "err", err, ) return @@ -526,42 +498,40 @@ func (n *Node) handleScheduleBatch(force bool) { // nolint: gocyclo switch { case len(batch) > 0: // We have some transactions, schedule batch. - case force && len(roundResults.Messages) > 0: + case force && len(n.roundResults.Messages) > 0: // We have runtime message results (and batch timeout expired), schedule batch. case force && inMsgMeta.Size > 0: // We have queued incoming runtime messages (and batch timeout expired), schedule batch. - case rtState.LastNormalRound == rtState.GenesisBlock.Header.Round: + case n.rtState.LastNormalRound == n.rtState.GenesisBlock.Header.Round: // This is the runtime genesis, schedule batch. - case force && rtState.LastNormalHeight < epoch.GetEpochHeight(): + case force && n.rtState.LastNormalHeight < n.epoch.GetEpochHeight(): // No block in this epoch processed by runtime yet, schedule batch. default: // No need to schedule a batch. + n.logger.Debug("not scheduling, no transactions") return } - // If we are an executor and not a scheduler try proposing a timeout. - if !epoch.IsTransactionScheduler(n.commonNode.CurrentBlock.Header.Round) { - n.logger.Debug("considering to propose a timeout", - "round", n.commonNode.CurrentBlock.Header.Round, - "batch_size", len(batch), - "round_results", roundResults, - ) - - if err := n.proposeTimeoutLocked(n.roundCtx); err != nil { - n.logger.Error("error proposing a timeout", - "err", err, - ) - } + ctx, cancel := context.WithCancel(ctx) + done := make(chan struct{}) - // If we are not a transaction scheduler, we can't really schedule. - n.logger.Debug("not scheduling a batch, not a transaction scheduler") - return - } + n.transitionState(StateProcessingBatch{ + mode: protocol.ExecutionModeSchedule, + priority: n.priority, + batchStartTime: time.Now(), + cancelFn: cancel, + done: done, + }) - n.startRuntimeBatchSchedulingLocked(rtState, roundResults, rt, batch) + // Request the worker host to schedule a batch. This is done in a separate + // goroutine so that the runtime worker can continue processing events. + go func() { + defer close(done) + n.startSchedulingBatch(ctx, batch) + }() } -func (n *Node) schedulerStoreTransactions(ctx context.Context, blk *block.Block, inputWriteLog storage.WriteLog, inputRoot hash.Hash) error { +func (n *Node) storeTransactions(ctx context.Context, blk *block.Block, inputWriteLog storage.WriteLog, inputRoot hash.Hash) error { var emptyRoot hash.Hash emptyRoot.Empty() @@ -576,134 +546,80 @@ func (n *Node) schedulerStoreTransactions(ctx context.Context, blk *block.Block, }) } -func (n *Node) schedulerCreateProposalLocked(ctx context.Context, inputRoot hash.Hash, txHashes []hash.Hash) (*commitment.Proposal, error) { - blk := n.commonNode.CurrentBlock - - // Create new proposal. - proposal := &commitment.Proposal{ - NodeID: n.commonNode.Identity.NodeSigner.Public(), - Header: commitment.ProposalHeader{ - Round: blk.Header.Round + 1, - PreviousHash: blk.Header.EncodedHash(), - BatchHash: inputRoot, - }, - Batch: txHashes, - } - if err := proposal.Sign(n.commonNode.Identity.NodeSigner, blk.Header.Namespace); err != nil { - return nil, fmt.Errorf("failed to sign proposal header: %w", err) +func (n *Node) publishProposal(ctx context.Context, proposal *commitment.Proposal) error { + if err := proposal.Sign(n.commonNode.Identity.NodeSigner, n.commonNode.Runtime.ID()); err != nil { + return fmt.Errorf("failed to sign proposal header: %w", err) } n.logger.Debug("dispatching a new batch proposal", - "input_root", inputRoot, - "batch_size", len(txHashes), + "input_root", proposal.Header.BatchHash, + "batch_size", len(proposal.Batch), ) n.commonNode.P2P.Publish(ctx, n.committeeTopic, &p2p.CommitteeMessage{ - Epoch: n.commonNode.CurrentEpoch, + Epoch: n.blockInfo.Epoch, Proposal: proposal, }) + crash.Here(crashPointBatchPublishAfter) - return proposal, nil + + return nil } -func (n *Node) startRuntimeBatchSchedulingLocked( - rtState *roothash.RuntimeState, - roundResults *roothash.RoundResults, - rt host.RichRuntime, - batch []*txpool.TxQueueMeta, -) { - n.logger.Debug("asking runtime to schedule batch", - "initial_batch_size", len(batch), +func (n *Node) startSchedulingBatch(ctx context.Context, batch []*txpool.TxQueueMeta) { + // This method runs within its own goroutine and is always stopped before the runtime + // worker finishes. Therefore, it is safe to read local round variables (block info, ...). + n.logger.Debug("scheduling batch", + "batch_size", len(batch), ) - // Create batch processing context and channel for receiving the response. - ctx, cancel := context.WithCancel(n.roundCtx) - done := make(chan *processedBatch, 1) - - batchStartTime := time.Now() - n.transitionLocked(StateProcessingBatch{&unresolvedBatch{}, batchStartTime, cancel, done, protocol.ExecutionModeSchedule}) - - // Request the worker host to process a batch. This is done in a separate - // goroutine so that the committee node can continue processing blocks. - blk := n.commonNode.CurrentBlock - consensusBlk := n.commonNode.CurrentConsensusBlock - epoch := n.commonNode.CurrentEpoch - - go func() { - defer close(done) - - initialBatch := make([][]byte, 0, len(batch)) - for _, tx := range batch { - initialBatch = append(initialBatch, tx.Raw()) - } - - // Ask the runtime to execute the batch. - rsp, err := n.runtimeExecuteTxBatch( - ctx, - rt, - protocol.ExecutionModeSchedule, - epoch, - consensusBlk, - blk, - rtState, - roundResults, - hash.Hash{}, // IORoot is ignored as it is yet to be determined. - initialBatch, + initialBatch := make([][]byte, 0, len(batch)) + for _, tx := range batch { + initialBatch = append(initialBatch, tx.Raw()) + } + + // Ask the runtime to execute the batch. + rsp, err := n.runtimeExecuteTxBatch( + ctx, + n.rt, + protocol.ExecutionModeSchedule, + n.blockInfo.Epoch, + n.blockInfo.ConsensusBlock, + n.blockInfo.RuntimeBlock, + n.rtState, + n.roundResults, + hash.Hash{}, // IORoot is ignored as it is yet to be determined. + initialBatch, + ) + if err != nil { + n.logger.Error("runtime batch execution failed", + "err", err, ) - if err != nil { - n.logger.Error("runtime batch execution failed", - "err", err, - ) - return - } - - // Remove any rejected transactions. - n.commonNode.TxPool.HandleTxsUsed(rsp.TxRejectHashes) - // Mark any proposed transactions. - n.commonNode.TxPool.PromoteProposedBatch(rsp.TxHashes) - - // Submit response to the executor worker. - done <- &processedBatch{ - computed: &rsp.Batch, - txHashes: rsp.TxHashes, - txInputRoot: rsp.TxInputRoot, - txInputWriteLog: rsp.TxInputWriteLog, - } - }() -} - -// Guarded by n.commonNode.CrossNode. -func (n *Node) maybeStartProcessingBatchLocked(batch *unresolvedBatch) { - epoch := n.commonNode.Group.GetEpochSnapshot() + return + } - switch { - case epoch.IsExecutorWorker(): - // Worker, start processing immediately. - n.startProcessingBatchLocked(batch) - case epoch.IsExecutorBackupWorker(): - // Backup worker, wait for discrepancy event. - state, ok := n.state.(StateWaitingForBatch) - if ok && state.discrepancyDetected { - // We have already received a discrepancy event, start processing immediately. - n.logger.Info("already received a discrepancy event, start processing batch") - n.startProcessingBatchLocked(batch) - return - } + // Remove any rejected transactions. + n.commonNode.TxPool.HandleTxsUsed(rsp.TxRejectHashes) + // Mark any proposed transactions. + _, _ = n.commonNode.TxPool.PromoteProposedBatch(rsp.TxHashes) - // Immediately resolve the batch so that we are ready when needed. - switch resolvedBatch, err := n.resolveBatchLocked(batch, StateWaitingForEvent{batch}); { - case err != nil: - // TODO: We should indicate failure. - return - case resolvedBatch == nil: - // Missing transactions, batch processing will start once all are available. - return - } + // Create new proposal. + proposal := commitment.Proposal{ + NodeID: n.commonNode.Identity.NodeSigner.Public(), + Header: commitment.ProposalHeader{ + Round: n.blockInfo.RuntimeBlock.Header.Round + 1, + PreviousHash: n.blockInfo.RuntimeBlock.Header.EncodedHash(), + BatchHash: rsp.TxInputRoot, + }, + Batch: rsp.TxHashes, + } - n.transitionLocked(StateWaitingForEvent{batch: batch}) - default: - // Currently not a member of an executor committee, log. - n.logger.Debug("not an executor committee member, ignoring batch") + // Submit response to the executor worker. + n.processedBatchCh <- &processedBatch{ + proposal: &proposal, + priority: n.priority, + computed: &rsp.Batch, + txInputWriteLog: rsp.TxInputWriteLog, } } @@ -901,146 +817,88 @@ func (n *Node) runtimeExecuteTxBatch( return rsp.RuntimeExecuteTxBatchResponse, nil } -// Guarded by n.commonNode.CrossNode. -func (n *Node) startProcessingBatchLocked(batch *unresolvedBatch) { - if n.commonNode.CurrentBlock == nil { - panic("attempted to start processing batch with a nil block") - } - - // Try to resolve the batch first. - resolvedBatch, err := n.resolveBatchLocked(batch, StateWaitingForTxs{batch}) - switch { - case err != nil: - // TODO: We should indicate failure. - return - case resolvedBatch == nil: - // Missing transactions, we will be called again once all are available. - return - } - +func (n *Node) startProcessingBatch(ctx context.Context, proposal *commitment.Proposal, priority uint64, batch transaction.RawBatch) { + // This method runs within its own goroutine and is always stopped before the runtime + // worker finishes. Therefore, it is safe to read local round variables (block info, ...). n.logger.Debug("processing batch", - "batch_size", len(resolvedBatch), + "batch_size", len(batch), ) - // Create batch processing context and channel for receiving the response. - ctx, cancel := context.WithCancel(n.roundCtx) - done := make(chan *processedBatch, 1) - - batchStartTime := time.Now() - n.transitionLocked(StateProcessingBatch{batch, batchStartTime, cancel, done, protocol.ExecutionModeExecute}) - - rt := n.commonNode.GetHostedRuntime() - if rt == nil { - // This should not happen as we only register to be an executor worker - // once the hosted runtime is ready. - n.logger.Error("received a batch while hosted runtime is not yet initialized") - n.abortBatchLocked(errRuntimeAborted) + // Optionally start local storage replication in parallel to batch dispatch. + replicateCh := n.startLocalStorageReplication(ctx, n.blockInfo.RuntimeBlock, proposal.Header.BatchHash, batch) + + // Ask the runtime to execute the batch. + rsp, err := n.runtimeExecuteTxBatch( + ctx, + n.rt, + protocol.ExecutionModeExecute, + n.blockInfo.Epoch, + n.blockInfo.ConsensusBlock, + n.blockInfo.RuntimeBlock, + n.rtState, + n.roundResults, + proposal.Header.BatchHash, + batch, + ) + if err != nil { + n.logger.Error("runtime batch execution failed", + "err", err, + ) return } - // Request the worker host to process a batch. This is done in a separate - // goroutine so that the committee node can continue processing blocks. - blk := n.commonNode.CurrentBlock - consensusBlk := n.commonNode.CurrentConsensusBlock - height := n.commonNode.CurrentBlockHeight - epoch := n.commonNode.CurrentEpoch - - go func() { - defer close(done) - - state, roundResults, err := n.getRtStateAndRoundResults(ctx, height) - if err != nil { - n.logger.Error("failed to query runtime state and last round results", - "err", err, - "height", height, - "round", blk.Header.Round, - ) - return - } - - // Optionally start local storage replication in parallel to batch dispatch. - replicateCh := n.startLocalStorageReplication(ctx, blk, batch.hash(), resolvedBatch) - - // Ask the runtime to execute the batch. - rsp, err := n.runtimeExecuteTxBatch( - ctx, - rt, - protocol.ExecutionModeExecute, - epoch, - consensusBlk, - blk, - state, - roundResults, - batch.hash(), - resolvedBatch, - ) + // Wait for replication to complete before proposing a batch to ensure that we can cleanly + // apply any updates. + select { + case <-ctx.Done(): + return + case err = <-replicateCh: if err != nil { - n.logger.Error("runtime batch execution failed", + n.logger.Error("local storage replication failed", "err", err, ) return } + } - // Wait for replication to complete before proposing a batch to ensure that we can cleanly - // apply any updates. - select { - case <-ctx.Done(): - return - case err = <-replicateCh: - if err != nil { - n.logger.Error("local storage replication failed", - "err", err, - ) - return - } - } - - // Submit response to the executor worker. - done <- &processedBatch{ - computed: &rsp.Batch, - raw: resolvedBatch, - txHashes: rsp.TxHashes, - } - }() + // Submit response to the round worker. + n.processedBatchCh <- &processedBatch{ + proposal: proposal, + priority: priority, + computed: &rsp.Batch, + raw: batch, + } } -// Guarded by n.commonNode.CrossNode. -func (n *Node) abortBatchLocked(reason error) { - state, ok := n.state.(StateProcessingBatch) - if !ok { - // We can only abort if a batch is being processed. - return - } +func (n *Node) abortBatch(state *StateProcessingBatch) { + n.logger.Warn("aborting processing batch") - n.logger.Warn("aborting batch", - "reason", reason, - ) + // Stop processing. + state.Cancel() - // Cancel the batch processing context and wait for it to finish. - state.cancel() + // Discard the result if there was any. + select { + case <-n.processedBatchCh: + default: + } crash.Here(crashPointBatchAbortAfter) abortedBatchCount.With(n.getMetricLabels()).Inc() - // After the batch has been aborted, we must wait for the round to be - // finalized. - n.transitionLocked(StateWaitingForFinalize{ - batchStartTime: state.batchStartTime, - }) } func (n *Node) proposeBatch( roundCtx context.Context, lastHeader *block.Header, - unresolved *unresolvedBatch, processed *processedBatch, ) { crash.Here(crashPointBatchProposeBefore) batch := processed.computed - epoch := n.commonNode.Group.GetEpochSnapshot() n.logger.Debug("proposing batch", + "scheduler_id", processed.proposal.NodeID, + "node_id", n.commonNode.Identity.NodeSigner.Public(), "batch_size", len(processed.raw), "io_root", *batch.Header.IORoot, "state_root", *batch.Header.StateRoot, @@ -1054,22 +912,17 @@ func (n *Node) proposeBatch( ec := &commitment.ExecutorCommitment{ NodeID: n.commonNode.Identity.NodeSigner.Public(), Header: commitment.ExecutorCommitmentHeader{ + SchedulerID: processed.proposal.NodeID, Header: batch.Header, RAKSignature: &rakSig, }, } // If we are the transaction scheduler also include all the emitted messages. - if epoch.IsTransactionScheduler(lastHeader.Round) { + if ec.NodeID.Equal(ec.Header.SchedulerID) { ec.Messages = batch.Messages } - var inputRoot hash.Hash - switch { - case unresolved.proposal == nil: - inputRoot = processed.txInputRoot - default: - inputRoot = unresolved.hash() - } + inputRoot := processed.proposal.Header.BatchHash // Commit I/O and state write logs to storage. storageErr := func() error { @@ -1116,49 +969,56 @@ func (n *Node) proposeBatch( } // Submit commitment. - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() - // Make sure we are still in the right state/round. state, ok := n.state.(StateProcessingBatch) - if !ok || lastHeader.Round != n.commonNode.CurrentBlock.Header.Round { + if !ok || lastHeader.Round != n.blockInfo.RuntimeBlock.Header.Round { n.logger.Error("new state or round since started proposing batch", "state", state, - "round", n.commonNode.CurrentBlock.Header.Round, + "round", n.blockInfo.RuntimeBlock.Header.Round, "expected_round", lastHeader.Round, ) return } + n.logger.Error("sign and submit the commitment", + "commit", ec, + ) + if err := n.signAndSubmitCommitment(roundCtx, ec); err != nil { n.logger.Error("failed to sign and submit the commitment", "commit", ec, "err", err, ) - n.abortBatchLocked(err) + n.abortBatch(&state) + return + } + + n.submitted[processed.priority] = struct{}{} + + if storageErr != nil { + n.abortBatch(&state) + n.transitionState(StateWaitingForBatch{}) return } // Due to backwards compatibility with runtimes that don't provide transaction hashes as output // we need to manually compute them here. - if len(processed.raw) > 0 && len(processed.txHashes) == 0 { - processed.txHashes = make([]hash.Hash, 0, len(processed.raw)) + txHashes := processed.proposal.Batch + if len(processed.raw) > 0 && len(txHashes) == 0 { + txHashes = make([]hash.Hash, 0, len(processed.raw)) for _, tx := range processed.raw { - processed.txHashes = append(processed.txHashes, hash.NewFromBytes(tx)) + txHashes = append(txHashes, hash.NewFromBytes(tx)) } } - switch storageErr { - case nil: - n.transitionLocked(StateWaitingForFinalize{ - batchStartTime: state.batchStartTime, - proposedIORoot: *ec.Header.Header.IORoot, - txHashes: processed.txHashes, - }) - default: - n.abortBatchLocked(storageErr) + n.proposedBatch = &proposedBatch{ + batchStartTime: state.batchStartTime, + proposedIORoot: *ec.Header.Header.IORoot, + txHashes: txHashes, } + n.transitionState(StateWaitingForBatch{}) + crash.Here(crashPointBatchProposeAfter) } @@ -1189,30 +1049,124 @@ func (n *Node) signAndSubmitCommitment(roundCtx context.Context, ec *commitment. return nil } -// Guarded by n.commonNode.CrossNode. -func (n *Node) handleProposalLocked(batch *unresolvedBatch) error { - n.logger.Debug("handling a new batch proposal", - "proposer", batch.proposal.NodeID, - "round", batch.proposal.Header.Round, +func (n *Node) processProposal(ctx context.Context, proposal *commitment.Proposal, priority uint64, discrepancy bool) { + n.logger.Debug("trying to process a proposal", + "scheduler", proposal.NodeID, + "round", proposal.Header.Round, + "priority", priority, + "discrepancy", discrepancy, ) - // TODO: Handle proposal equivocation. + // Check if we are in a suitable state for processing a proposal. + switch n.state.(type) { + case StateWaitingForBatch: + default: + return + } + + // Process only once. + if _, ok := n.submitted[priority]; ok { + n.logger.Debug("not processing, commitment already submitted") + return + } - if _, ok := n.state.(StateWaitingForBatch); !ok { - // Currently not waiting for batch. - } else if epoch := n.commonNode.Group.GetEpochSnapshot(); !epoch.IsExecutorMember() { - // Currently not an executor committee member. - } else { - // Maybe process if we have the correct block. - currentHash := n.commonNode.CurrentBlock.Header.EncodedHash() - if currentHash.Equal(&batch.proposal.Header.PreviousHash) { - n.maybeStartProcessingBatchLocked(batch) - return nil // Forward proposal. + switch discrepancy { + case true: + // Only backup executor workers are permitted to process batches. + if !n.epoch.IsExecutorBackupWorker() { + n.logger.Debug("not processing, not a backup executor") + return + } + case false: + // All workers are allowed to process batches. The only difference is that the backup + // execution workers will wait for a discrepancy event before beginning execution. + } + + n.logger.Debug("attempting to resolve batch") + + // Try to resolve the batch first. + // TODO: Add metrics for how long it takes to receive the complete batch. + resolvedBatch, missingTxs := n.commonNode.TxPool.PromoteProposedBatch(proposal.Batch) + + // Compute batch size. + batchSize := uint64(len(proposal.Batch)) + bytes := uint64(0) + for _, tx := range resolvedBatch { + if tx == nil { + continue + } + bytes += uint64(tx.Size()) + } + + // Submit failure if the batch is invalid. + // The scheduler is violating the protocol and should be punished. + maxBatchSize := n.blockInfo.ActiveDescriptor.TxnScheduler.MaxBatchSize + maxBytes := n.blockInfo.ActiveDescriptor.TxnScheduler.MaxBatchSizeBytes + if batchSize > maxBatchSize || maxBytes > 0 && bytes > maxBytes { + n.transitionStateToProcessingFailure(ctx, proposal, priority, bytes, maxBytes, batchSize, maxBatchSize) + return + } + + // Prepare the batch. If some transactions are missing, they will be filled latter. + batch := make(transaction.RawBatch, 0, len(resolvedBatch)) + for _, tx := range resolvedBatch { + switch tx { + case nil: + batch = append(batch, nil) + default: + batch = append(batch, tx.Raw()) } } - // When checks fail, add proposal into the queue of pending proposals for later retry. - return n.addPendingProposalLocked(batch) + // Missing transactions, we will wait until all are received. + if len(missingTxs) > 0 { + n.logger.Debug("some transactions are missing", "num_missing", len(missingTxs)) + + txHashes := maps.Keys(missingTxs) + + subCtx, cancelFn := context.WithCancel(ctx) + done := make(chan struct{}) + + n.transitionState(StateWaitingForTxs{ + proposal: proposal, + priority: priority, + batch: batch, + txs: missingTxs, + bytes: bytes, + maxBytes: maxBytes, + batchSize: batchSize, + maxBatchSize: maxBatchSize, + cancelFn: cancelFn, + done: done, + }) + + go func() { + defer close(done) + n.requestMissingTransactions(subCtx, txHashes) + }() + + return + } + + // TODO: Handle proposal equivocation. + + // Maybe process if we have the correct block. + currentHash := n.blockInfo.RuntimeBlock.Header.EncodedHash() + if !currentHash.Equal(&proposal.Header.PreviousHash) { + return + } + + // The backup workers should process only if the discrepancy was detected. + if !n.epoch.IsExecutorWorker() && n.epoch.IsExecutorBackupWorker() && !discrepancy { + n.transitionState(StateWaitingForEvent{ + proposal: proposal, + priority: priority, + batch: batch, + }) + return + } + + n.transitionStateToProcessing(ctx, proposal, priority, batch) } // nudgeAvailabilityLocked checks whether the executor worker should declare itself available. @@ -1320,82 +1274,142 @@ func (n *Node) HandleRuntimeHostEventLocked(ev *host.Event) { n.nudgeAvailabilityLocked(true) } -func (n *Node) handleProcessedBatch(batch *processedBatch, processingCh chan *processedBatch) { - n.commonNode.CrossNode.Lock() - - // To avoid stale events, check if the stored state is still valid. - // XXX: processingCh not changing ensures we are in the same state and not - // in a "new" processing batch state. This also ensures that the round did - // not change. +func (n *Node) handleProcessedBatch(ctx context.Context, batch *processedBatch) { state, ok := n.state.(StateProcessingBatch) - if !ok || state.done != processingCh { - n.commonNode.CrossNode.Unlock() + if !ok { + // Should not be possible, as we always drain the channel once we transition + // to a different state. + n.logger.Error("failed to handle processed batch, invalid state", + "state", n.state, + ) return } - roundCtx := n.roundCtx - lastHeader := n.commonNode.CurrentBlock.Header + lastHeader := n.blockInfo.RuntimeBlock.Header - switch { - case batch == nil || batch.computed == nil: - // There was an issue during batch processing. - case state.mode == protocol.ExecutionModeSchedule: - // Scheduling was processed successfully. + // Check if there was an issue during batch processing. + if batch.computed == nil { + n.logger.Warn("worker has aborted batch processing") + + n.abortBatch(&state) + n.transitionState(StateWaitingForBatch{}) + + commit := &commitment.ExecutorCommitment{ + NodeID: n.commonNode.Identity.NodeSigner.Public(), + Header: commitment.ExecutorCommitmentHeader{ + SchedulerID: batch.proposal.NodeID, + Header: commitment.ComputeResultsHeader{ + Round: lastHeader.Round + 1, + PreviousHash: lastHeader.EncodedHash(), + }, + }, + } + commit.Header.SetFailure(commitment.FailureUnknown) + + n.logger.Debug("submitting failure indicating commitment", + "commitment", commit, + ) + if err := n.signAndSubmitCommitment(ctx, commit); err != nil { + n.logger.Error("failed to sign and submit the commitment", + "commit", commit, + "err", err, + ) + return + } + + n.submitted[batch.priority] = struct{}{} + return + } + + // Check if scheduling was processed successfully. + if state.mode == protocol.ExecutionModeSchedule { n.logger.Info("runtime has finished scheduling a batch", - "input_root", batch.txInputRoot, - "tx_hashes", batch.txHashes, + "input_root", batch.proposal.Header.BatchHash, + "tx_hashes", batch.proposal.Batch, ) - err := n.schedulerStoreTransactions(roundCtx, n.commonNode.CurrentBlock, batch.txInputWriteLog, batch.txInputRoot) + err := n.storeTransactions(ctx, n.blockInfo.RuntimeBlock, batch.txInputWriteLog, batch.proposal.Header.BatchHash) if err != nil { n.logger.Error("failed to store transaction", "err", err, ) - break + return } - // Create and submit a proposal. - _, err = n.schedulerCreateProposalLocked(roundCtx, batch.txInputRoot, batch.txHashes) + // Sign and submit the proposal to P2P network. + err = n.publishProposal(ctx, batch.proposal) if err != nil { - n.logger.Error("failed to create proposal", + n.logger.Error("failed to sign and publish proposal", "err", err, ) - break + return } - fallthrough - default: - // Batch was processed successfully. - stateBatch := state.batch - n.commonNode.CrossNode.Unlock() - n.logger.Info("worker has finished processing a batch") - n.proposeBatch(roundCtx, &lastHeader, stateBatch, batch) + } + + // Batch was processed successfully. + n.logger.Info("worker has finished processing a batch") + n.proposeBatch(ctx, &lastHeader, batch) +} + +func (n *Node) handleRoundStarted() { + n.logger.Debug("starting round worker", + "round", n.blockInfo.RuntimeBlock.Header.Round+1, + ) + + n.logger.Info("considering the round finalized", + "round", n.blockInfo.RuntimeBlock.Header.Round, + "header_hash", n.blockInfo.RuntimeBlock.Header.EncodedHash(), + "header_type", n.blockInfo.RuntimeBlock.Header.HeaderType, + ) + if n.blockInfo.RuntimeBlock.Header.HeaderType != block.Normal { return } - defer n.commonNode.CrossNode.Unlock() + if n.proposedBatch == nil { + return + } - // Unsuccessful batch processing. - n.logger.Warn("worker has aborted batch processing") - commit := &commitment.ExecutorCommitment{ - NodeID: n.commonNode.Identity.NodeSigner.Public(), - Header: commitment.ExecutorCommitmentHeader{ - Header: commitment.ComputeResultsHeader{ - Round: lastHeader.Round + 1, - PreviousHash: lastHeader.EncodedHash(), - }, - }, + if !n.blockInfo.RuntimeBlock.Header.IORoot.Equal(&n.proposedBatch.proposedIORoot) { + n.logger.Error("proposed batch was not finalized", + "header_io_root", n.blockInfo.RuntimeBlock.Header.IORoot, + "proposed_io_root", n.proposedBatch.proposedIORoot, + "header_type", n.blockInfo.RuntimeBlock.Header.HeaderType, + "batch_size", len(n.proposedBatch.txHashes), + ) + return } - commit.Header.SetFailure(commitment.FailureUnknown) - n.logger.Debug("submitting failure indicating commitment", - "commitment", commit, + // Record time taken for successfully processing a batch. + batchProcessingTime.With(n.getMetricLabels()).Observe(time.Since(n.proposedBatch.batchStartTime).Seconds()) + + n.logger.Debug("removing processed batch from queue", + "batch_size", len(n.proposedBatch.txHashes), + "io_root", n.blockInfo.RuntimeBlock.Header.IORoot, ) - if err := n.signAndSubmitCommitment(roundCtx, commit); err != nil { - n.logger.Error("failed to sign and submit the commitment", - "commit", commit, - "err", err, - ) + + // Remove processed transactions from queue. + n.commonNode.TxPool.HandleTxsUsed(n.proposedBatch.txHashes) +} + +func (n *Node) handleRoundEnded() { + n.logger.Debug("stopping round worker", + "round", n.blockInfo.RuntimeBlock.Header.Round+1, + ) + + switch state := n.state.(type) { + case StateWaitingForBatch: + // Nothing to do here. + return + case StateWaitingForTxs: + // Stop waiting for transactions. + n.logger.Warn("considering the round failed due to missing transactions") + state.Cancel() + case StateWaitingForEvent: + // Block finalized without the need for a backup worker. + n.logger.Info("considering the round finalized without backup worker") + case StateProcessingBatch: + n.abortBatch(&state) } - n.abortBatchLocked(errRuntimeAborted) + n.transitionState(StateWaitingForBatch{}) } func (n *Node) worker() { @@ -1410,18 +1424,20 @@ func (n *Node) worker() { return } - n.logger.Info("starting committee node") + n.logger.Info("starting worker") + + var ( + err error + txSub pubsub.ClosableSubscription + ecSub pubsub.ClosableSubscription + ) // Subscribe to notifications of new transactions being available in the pool. - txSub, txCh := n.commonNode.TxPool.WatchCheckedTransactions() + txSub, n.txCh = n.commonNode.TxPool.WatchCheckedTransactions() defer txSub.Close() - // Subscribe to scheduler notifications. - schedSub, schedCh := n.commonNode.TxPool.WatchScheduler() - defer schedSub.Close() - // Subscribe to gossiped executor commitments. - ecCh, ecSub, err := n.commonNode.Consensus.RootHash().WatchExecutorCommitments(n.ctx) + n.ecCh, ecSub, err = n.commonNode.Consensus.RootHash().WatchExecutorCommitments(n.ctx) if err != nil { n.logger.Error("failed to subscribe to executor commitments", "err", err, @@ -1447,17 +1463,27 @@ func (n *Node) worker() { } }() + // (Re)Start the runtime worker every time a runtime block is finalized. + var ( + wg sync.WaitGroup + bi *runtime.BlockInfo + ) for { - // Check if we are currently processing a batch. In this case, we also - // need to select over the result channel. - var processingDoneCh chan *processedBatch - func() { - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() + wg.Add(1) + defer wg.Wait() - if stateProcessing, ok := n.state.(StateProcessingBatch); ok { - processingDoneCh = stateProcessing.done + ctx, cancel := context.WithCancel(n.ctx) + defer cancel() + + go func() { + defer wg.Done() + n.roundWorker(ctx, bi) + }() + + select { + case <-n.stopCh: + case bi = <-n.blockInfoCh: } }() @@ -1465,20 +1491,154 @@ func (n *Node) worker() { case <-n.stopCh: n.logger.Info("termination requested") return - case batch := <-processingDoneCh: - // Batch processing has finished. - n.handleProcessedBatch(batch, processingDoneCh) - case force := <-schedCh: - // Attempt scheduling. - n.handleScheduleBatch(force) - case txs := <-txCh: + default: + } + } +} + +func (n *Node) roundWorker(ctx context.Context, bi *runtime.BlockInfo) { + if bi == nil { + return + } + n.blockInfo = bi + round := bi.RuntimeBlock.Header.Round + 1 + + n.handleRoundStarted() + defer n.handleRoundEnded() + + // Clear last proposal. + n.proposedBatch = nil + + // Clear proposal queue. + n.commonNode.TxPool.ClearProposedBatch() + + // Prune proposals. + n.proposals.Prune(round) + + // Need to be an executor committee member. + n.epoch = n.commonNode.Group.GetEpochSnapshot() + if !n.epoch.IsExecutorMember() { + n.logger.Debug("skipping round, not an executor member", + "round", round, + ) + return + } + + // This should never fail as we only register to be an executor worker + // once the hosted runtime is ready. + n.rt = n.commonNode.GetHostedRuntime() + if n.rt == nil { + n.logger.Error("skipping round, hosted runtime is not yet initialized") + return + } + + // Fetch state and round results upfront. + var err error + n.rtState, n.roundResults, err = n.getRtStateAndRoundResults(ctx, bi.ConsensusBlock.Height) + if err != nil { + n.logger.Debug("skipping round, failed to fetch state and round results", + "err", err, + ) + return + } + + // Reset propose timeout timer. + n.timeoutActive = false + n.timeoutTimer = time.NewTimer(time.Hour) + defer n.timeoutTimer.Stop() + + // Prepare flush timer for the primary transaction scheduler. + flush := false + flushTimer := time.NewTimer(bi.ActiveDescriptor.TxnScheduler.BatchFlushTimeout) + defer flushTimer.Stop() + + // Compute node's priority when scheduling transactions. + id := n.commonNode.Identity.NodeSigner.Public() + committee := n.epoch.GetExecutorCommittee().Committee + n.priority = committee.SchedulingPriority(bi.RuntimeBlock.Header.Round, id) + n.logger.Debug("node is an executor member", + "round", round, + "priority", n.priority, + "executor_worker", n.epoch.IsExecutorWorker(), + "backup_executor_worker", n.epoch.IsExecutorBackupWorker(), + ) + + // Setting max priority to zero allows the primary transaction scheduler to propose immediately. + // The ticker dictates when the backup schedulers can propose. + var maxPriority uint64 + maxPriorityTicker := NewLinearPriorityTicker(uint64(bi.RuntimeBlock.Header.Timestamp), priorityDelay, n.priority) + maxPriorityTicker.Start() + defer maxPriorityTicker.Stop() + + // Remember if discrepancy was detected. + var discrepancy *discrepancyEvent + + // Reset submitted proposals/commitments. + n.submitted = make(map[uint64]struct{}) + + // Main loop. + for { + // Update state, propose or schedule. + switch discrepancy { + case nil: + proposal, priority, ok := n.proposals.Best(round, 0, maxPriority) + switch { + case ok && priority < n.priority: + // Try to process the best proposal. + n.updateState(ctx, 0, priority, false) + n.processProposal(ctx, proposal, priority, false) + case n.priority <= maxPriority: + // Try to schedule a batch. + n.updateState(ctx, 0, n.priority, false) + n.scheduleBatch(ctx, round, flush) + } + default: + n.updateState(ctx, discrepancy.priority, discrepancy.priority, true) + + proposal, priority, ok := n.proposals.Best(round, discrepancy.priority, discrepancy.priority) + switch { + case ok: + // Try to process the discrepant proposal. + n.processProposal(ctx, proposal, priority, true) + case n.priority == discrepancy.priority: + // Try to schedule a batch. + n.scheduleBatch(ctx, round, true) + } + } + + select { + case <-ctx.Done(): + n.logger.Debug("exiting round, context canceled") + return + case blk := <-n.consensusBlockCh: + // Check scheduler timeout. + n.handleNewConsensusBlock(ctx, blk) + case txs := <-n.txCh: // Check any queued transactions. n.handleNewCheckedTransactions(txs) - case ec := <-ecCh: + case discrepancy = <-n.discrepancyCh: + // Discrepancy has been detected. + n.handleDiscrepancy(ctx, discrepancy) + case ec := <-n.ecCh: // Process observed executor commitments. - n.handleObservedExecutorCommitment(ec) - case <-n.reselect: - // Recalculate select set. + n.handleObservedExecutorCommitment(ctx, ec) + case batch := <-n.processedBatchCh: + // Batch processing has finished. + n.handleProcessedBatch(ctx, batch) + case <-n.timeoutTimer.C: + // It is time to propose a timeout. + n.handleTimeout(ctx, maxPriority) + case maxPriority = <-maxPriorityTicker.C(): + // Increase priority and try to schedule/process again. + n.logger.Debug("max priority has changed", + "priority", maxPriority, + ) + case <-flushTimer.C: + // Force scheduling for primary transaction scheduler. + n.logger.Debug("scheduling is now forced") + flush = true + case <-n.reselectCh: + // Try again. } } } @@ -1500,16 +1660,20 @@ func NewNode( commonCfg: commonCfg, roleProvider: roleProvider, committeeTopic: committeeTopic, - pendingProposals: newPendingProposals(), + proposals: newPendingProposals(), + commitPool: &commitment.Pool{}, ctx: ctx, cancelCtx: cancel, stopCh: make(chan struct{}), quitCh: make(chan struct{}), initCh: make(chan struct{}), - state: StateNotReady{}, + state: StateWaitingForBatch{}, txSync: txsync.NewClient(commonNode.P2P, commonNode.ChainContext, commonNode.Runtime.ID()), stateTransitions: pubsub.NewBroker(false), - reselect: make(chan struct{}, 1), + consensusBlockCh: make(chan *consensus.Block, 1), + blockInfoCh: make(chan *runtime.BlockInfo, 1), + discrepancyCh: make(chan *discrepancyEvent, 1), + processedBatchCh: make(chan *processedBatch, 1), logger: logging.GetLogger("worker/executor/committee").With("runtime_id", commonNode.Runtime.ID()), } diff --git a/go/worker/compute/executor/committee/p2p.go b/go/worker/compute/executor/committee/p2p.go index dcac5cc9f71..b1809232643 100644 --- a/go/worker/compute/executor/committee/p2p.go +++ b/go/worker/compute/executor/committee/p2p.go @@ -3,6 +3,7 @@ package committee import ( "context" "fmt" + "math" "github.com/oasisprotocol/oasis-core/go/common/cbor" "github.com/oasisprotocol/oasis-core/go/common/crash" @@ -50,7 +51,7 @@ func (h *committeeMsgHandler) AuthorizeMessage(ctx context.Context, peerID signa return fmt.Errorf("executor committee is not yet known") } - if !committee.Peers[peerID] { + if _, ok := committee.Peers[peerID]; !ok { return p2pError.Permanent(fmt.Errorf("peer is not authorized to publish committee messages")) } return nil @@ -65,6 +66,7 @@ func (h *committeeMsgHandler) HandleMessage(ctx context.Context, peerID signatur if isOwn { return nil } + crash.Here(crashPointBatchReceiveAfter) proposal := cm.Proposal @@ -72,9 +74,11 @@ func (h *committeeMsgHandler) HandleMessage(ctx context.Context, peerID signatur // Before opening the signed dispatch message, verify that it was actually signed by the // current transaction scheduler. - if err := epoch.VerifyTxnSchedulerSigner(proposal.NodeID, proposal.Header.Round-1); err != nil { + committee := epoch.GetExecutorCommittee().Committee + priority := committee.SchedulingPriority(proposal.Header.Round-1, proposal.NodeID) + if priority == math.MaxUint64 { // Not signed by the transaction scheduler for the round, do not forward. - return errMsgFromNonTxnSched + return p2pError.Permanent(errMsgFromNonTxnSched) } // Transaction scheduler checks out, verify signature. @@ -82,14 +86,16 @@ func (h *committeeMsgHandler) HandleMessage(ctx context.Context, peerID signatur return p2pError.Permanent(err) } - return h.n.processProposal(ctx, proposal) + // Add to the queue. + if err := h.n.proposals.Add(proposal, priority); err != nil { + return err + } + + // Notify the worker about the new proposal. + h.n.reselect() + + return nil default: return p2pError.ErrUnhandledMessage } } - -// HandlePeerTx implements NodeHooks. -func (n *Node) HandlePeerTx(ctx context.Context, tx []byte) error { - // Nothing to do here. - return nil -} diff --git a/go/worker/compute/executor/committee/proposals.go b/go/worker/compute/executor/committee/proposals.go index 96c5b911bd3..948c1ed7bf8 100644 --- a/go/worker/compute/executor/committee/proposals.go +++ b/go/worker/compute/executor/committee/proposals.go @@ -2,95 +2,124 @@ package committee import ( "fmt" + "sync" "github.com/google/btree" p2pError "github.com/oasisprotocol/oasis-core/go/p2p/error" + "github.com/oasisprotocol/oasis-core/go/roothash/api/commitment" ) // maxPendingProposals is the maximum number of pending proposals that can be queued. -const maxPendingProposals = 16 +const maxPendingProposals = 32 -// pendingProposals is a priority queue of pending proposals, ordered by round. -type pendingProposals struct { - q *btree.BTreeG[*unresolvedBatch] +type proposalInfo struct { + proposal *commitment.Proposal + priority uint64 } -func proposalLessFunc(a, b *unresolvedBatch) bool { +// proposalQueue is a priority queue of pending proposals, ordered by round and priority. +type proposalQueue struct { + sync.RWMutex + + q *btree.BTreeG[*proposalInfo] + + round uint64 +} + +func proposalLessFunc(a, b *proposalInfo) bool { + if a.proposal.Header.Round == b.proposal.Header.Round { + return a.priority < b.priority + } return a.proposal.Header.Round < b.proposal.Header.Round } -func newPendingProposals() *pendingProposals { - return &pendingProposals{ +func newPendingProposals() *proposalQueue { + return &proposalQueue{ q: btree.NewG(2, proposalLessFunc), } } -// addPendingProposalLocked adds a new pending proposal that MUST HAVE already undergone basic -// validity checks and is therefore considered a valid proposal for the given round, but the node's +// Best returns the best proposal for the given round with priority within given bounds. +func (q *proposalQueue) Best(round uint64, minPriority uint64, maxPriority uint64) (*commitment.Proposal, uint64, bool) { + q.RLock() + defer q.RUnlock() + + var ( + proposal *commitment.Proposal + priority uint64 + ok bool + ) + + q.q.Ascend(func(pi *proposalInfo) bool { + switch { + case pi.proposal.Header.Round < round: + return true + case pi.proposal.Header.Round > round: + return false + case pi.priority < minPriority: + return true + case pi.priority > maxPriority: + return false + default: + proposal = pi.proposal + priority = pi.priority + ok = true + return false + } + }) + + return proposal, priority, ok +} + +// Add adds a new pending proposal that MUST HAVE already undergone basic validity checks +// and is therefore considered a valid proposal for the given round, but the node's // local consensus view may not yet be ready to process the proposal. -// -// Must be called with the n.commonNode.CrossNode lock held. -func (n *Node) addPendingProposalLocked(batch *unresolvedBatch) error { - currentRound := n.commonNode.CurrentBlock.Header.Round - round := batch.proposal.Header.Round +func (q *proposalQueue) Add(proposal *commitment.Proposal, priority uint64) error { + q.Lock() + defer q.Unlock() // Drop any past proposals. - if round <= currentRound { + if proposal.Header.Round < q.round { return p2pError.Permanent(fmt.Errorf("proposal round is in the past")) // Do not forward. } - n.pendingProposals.q.ReplaceOrInsert(batch) + info := proposalInfo{ + proposal: proposal, + priority: priority, + } + q.q.ReplaceOrInsert(&info) - // In case of overflows, remove the round that is the most in the future. - n.prunePendingProposalsLocked() - if n.pendingProposals.q.Len() >= maxPendingProposals { - removed, _ := n.pendingProposals.q.DeleteMax() - if removed == batch { - return p2pError.Permanent(fmt.Errorf("proposal queue overflow")) // Do not forward. - } + // In case of overflows, remove the proposal that is the most in the future. + if q.q.Len() <= maxPendingProposals { + return nil + } + removed, _ := q.q.DeleteMax() + if removed == &info { + return fmt.Errorf("proposal queue overflow") } return nil } -// prunePendingProposalsLocked prunes any proposals which are not valid anymore. -// -// Must be called with the n.commonNode.CrossNode lock held. -func (n *Node) prunePendingProposalsLocked() { - currentRound := n.commonNode.CurrentBlock.Header.Round +// Prune prunes any proposals which are not valid anymore. +func (q *proposalQueue) Prune(round uint64) { + q.Lock() + defer q.Unlock() for { - batch, ok := n.pendingProposals.q.Min() + info, ok := q.q.Min() if !ok { break } - if batch.proposal.Header.Round > currentRound { + if info.proposal.Header.Round >= round { // All further proposals are valid. break } // Remove invalid proposals. - n.pendingProposals.q.DeleteMin() - } -} - -// handlePendingProposalsLocked attempts to handle any pending proposals. At most one proposal is -// handled. -// -// Must be called with the n.commonNode.CrossNode lock held. -func (n *Node) handlePendingProposalsLocked() { - // Prune any invalid pending proposals. - n.prunePendingProposalsLocked() - - // Dequeue the next proposal. - batch, ok := n.pendingProposals.q.DeleteMin() - if !ok { - return + q.q.DeleteMin() } - // Ignoring the error is fine, because the proposal is either handled (no error) or added - // back to the queue (no error since an overflow cannot happen given we just removed it). - // Since we checked above that the proposal is valid there is no other option. - _ = n.handleProposalLocked(batch) + q.round = round } diff --git a/go/worker/compute/executor/committee/state.go b/go/worker/compute/executor/committee/state.go index ff253250ba8..112b58e0e8a 100644 --- a/go/worker/compute/executor/committee/state.go +++ b/go/worker/compute/executor/committee/state.go @@ -5,6 +5,7 @@ import ( "time" "github.com/oasisprotocol/oasis-core/go/common/crypto/hash" + "github.com/oasisprotocol/oasis-core/go/roothash/api/commitment" "github.com/oasisprotocol/oasis-core/go/runtime/host/protocol" "github.com/oasisprotocol/oasis-core/go/runtime/transaction" storage "github.com/oasisprotocol/oasis-core/go/storage/api" @@ -14,32 +15,21 @@ import ( type StateName string const ( - // NotReady is the name of StateNotReady. - NotReady StateName = "NotReady" // WaitingForBatch is the name of StateWaitingForBatch. WaitingForBatch = "WaitingForBatch" - // WaitingForEvent is the name of StateWaitingForEvent. - WaitingForEvent = "WaitingForEvent" // WaitingForTxs is the name of the StateWaitingForTxs. WaitingForTxs = "WaitingForTxs" + // WaitingForEvent is the name of StateWaitingForEvent. + WaitingForEvent = "WaitingForEvent" // ProcessingBatch is the name of StateProcessingBatch. ProcessingBatch = "ProcessingBatch" - // WaitingForFinalize is the name of StateWaitingForFinalize. - WaitingForFinalize = "WaitingForFinalize" ) // Valid state transitions. var validStateTransitions = map[StateName][]StateName{ - // Transitions from NotReady state. - NotReady: { - // Epoch transition occurred and we are not in the committee. - NotReady, - // Epoch transition occurred and we are in the committee. - WaitingForBatch, - }, - // Transitions from WaitingForBatch state. WaitingForBatch: { + // Waiting batch, e.g. round ended. WaitingForBatch, // Received batch, current block is up to date. ProcessingBatch, @@ -47,43 +37,29 @@ var validStateTransitions = map[StateName][]StateName{ WaitingForEvent, // Received batch, waiting for missing transactions. WaitingForTxs, - // Epoch transition occurred and we are no longer in the committee. - NotReady, }, - // Transitions from WaitingForEvent state. - WaitingForEvent: { - // Abort: seen newer block while waiting for event. + WaitingForTxs: { + // Received batch with higher priority or round ended. WaitingForBatch, - // Discrepancy event received. + // Received all missing transactions, waiting for discrepancy event. + WaitingForEvent, + // Received all missing transactions (and discrepancy event). ProcessingBatch, - // Discrepancy event received, waiting for transactions. - WaitingForTxs, - // Epoch transition occurred and we are no longer in the committee. - NotReady, }, - WaitingForTxs: { - // Abort: seen newer block while waiting for missing transactions. + // Transitions from WaitingForEvent state. + WaitingForEvent: { + // Received batch with higher priority or round ended. WaitingForBatch, - // Received all missing transactions. + // Received discrepancy event. ProcessingBatch, - // Epoch transition occurred and we are no longer in the committee. - NotReady, }, // Transitions from ProcessingBatch state. ProcessingBatch: { - // Batch has been successfully processed or has been aborted. - WaitingForFinalize, - }, - - // Transitions from WaitingForFinalize state. - WaitingForFinalize: { - // Round has been finalized. + // Received batch with higher priority or round ended. WaitingForBatch, - // Epoch transition occurred and we are no longer in the committee. - NotReady, }, } @@ -93,25 +69,8 @@ type NodeState interface { Name() StateName } -// StateNotReady is the not ready state. -type StateNotReady struct{} - -// Name returns the name of the state. -func (s StateNotReady) Name() StateName { - return NotReady -} - -// String returns a string representation of the state. -func (s StateNotReady) String() string { - return string(s.Name()) -} - // StateWaitingForBatch is the waiting for batch state. -type StateWaitingForBatch struct { - // Whether a discrepancy has been detected in case the node is a backup worker and the event has - // been received before the batch. - discrepancyDetected bool -} +type StateWaitingForBatch struct{} // Name returns the name of the state. func (s StateWaitingForBatch) Name() StateName { @@ -125,8 +84,10 @@ func (s StateWaitingForBatch) String() string { // StateWaitingForEvent is the waiting for event state. type StateWaitingForEvent struct { - // Batch that is being processed. - batch *unresolvedBatch + proposal *commitment.Proposal + priority uint64 + + batch transaction.RawBatch } // Name returns the name of the state. @@ -140,8 +101,19 @@ func (s StateWaitingForEvent) String() string { } type StateWaitingForTxs struct { - // Batch that is being processed. - batch *unresolvedBatch + proposal *commitment.Proposal + priority uint64 + + batch transaction.RawBatch + txs map[hash.Hash]int + + bytes uint64 + maxBytes uint64 + batchSize uint64 + maxBatchSize uint64 + + cancelFn context.CancelFunc + done chan struct{} } // Name returns the name of the state. @@ -154,27 +126,24 @@ func (s StateWaitingForTxs) String() string { return string(s.Name()) } +// Cancel invokes the cancellation function and waits for the fetching to actually stop. +func (s StateWaitingForTxs) Cancel() { + s.cancelFn() + <-s.done +} + // StateProcessingBatch is the processing batch state. type StateProcessingBatch struct { - // Batch that is being processed. - batch *unresolvedBatch + priority uint64 + + // Execution mode. + mode protocol.ExecutionMode // Timing for this batch. batchStartTime time.Time // Function for cancelling batch processing. cancelFn context.CancelFunc // Channel which will provide the result. - done chan *processedBatch - // Execution mode. - mode protocol.ExecutionMode -} - -type processedBatch struct { - computed *protocol.ComputedBatch - raw transaction.RawBatch - - txHashes []hash.Hash - txInputRoot hash.Hash - txInputWriteLog storage.WriteLog + done chan struct{} } // Name returns the name of the state. @@ -187,26 +156,24 @@ func (s StateProcessingBatch) String() string { return string(s.Name()) } -func (s *StateProcessingBatch) cancel() { - // Invoke the cancellation function and wait for the processing - // to actually stop. - (s.cancelFn)() +// Cancel invokes the cancellation function and waits for the processing to actually stop. +func (s *StateProcessingBatch) Cancel() { + s.cancelFn() <-s.done } -// StateWaitingForFinalize is the waiting for finalize state. -type StateWaitingForFinalize struct { - batchStartTime time.Time - proposedIORoot hash.Hash - txHashes []hash.Hash -} +type processedBatch struct { + proposal *commitment.Proposal + priority uint64 -// Name returns the name of the state. -func (s StateWaitingForFinalize) Name() StateName { - return WaitingForFinalize + computed *protocol.ComputedBatch + raw transaction.RawBatch + + txInputWriteLog storage.WriteLog } -// String returns a string representation of the state. -func (s StateWaitingForFinalize) String() string { - return string(s.Name()) +type proposedBatch struct { + batchStartTime time.Time + proposedIORoot hash.Hash + txHashes []hash.Hash } diff --git a/go/worker/compute/executor/committee/ticker.go b/go/worker/compute/executor/committee/ticker.go new file mode 100644 index 00000000000..b04981c4f54 --- /dev/null +++ b/go/worker/compute/executor/committee/ticker.go @@ -0,0 +1,110 @@ +package committee + +import ( + "time" +) + +// PriorityTicker represents an interface for priority ticker functionality. +type PriorityTicker interface { + // Start starts the ticker. + Start() + + // Stop stops the ticker. + Stop() + + // C returns a channel on which priorities will be sent. + C() <-chan uint64 +} + +type linearPriorityTicker struct { + c chan uint64 + + stop chan struct{} + done chan struct{} + + timestamp int64 + duration time.Duration + maxPriority uint64 +} + +// NewLinearPriorityTicker returns a new linear priority ticker. +func NewLinearPriorityTicker(timestamp uint64, d time.Duration, maxPriority uint64) PriorityTicker { + if d <= 0 { + panic("duration has to be a positive number") + } + + return &linearPriorityTicker{ + c: make(chan uint64, 1), + stop: make(chan struct{}), + done: make(chan struct{}), + timestamp: int64(timestamp), + duration: d, + maxPriority: maxPriority, + } +} + +func (t *linearPriorityTicker) C() <-chan uint64 { + return t.c +} + +func (t *linearPriorityTicker) Start() { + go t.start() +} + +func (t *linearPriorityTicker) start() { + defer close(t.done) + + var priority uint64 + sendPriority := func() { + select { + case t.c <- priority: + default: + } + priority++ + } + + diff := time.Until(time.Unix(t.timestamp, 0)) + if diff < 0 { + priority = uint64(-diff / t.duration) + if priority > t.maxPriority { + priority = t.maxPriority + } + + sendPriority() + if priority > t.maxPriority { + return + } + + diff = t.duration + diff%t.duration + } + + timer := time.NewTimer(diff) + defer timer.Stop() + + select { + case <-timer.C: + case <-t.stop: + return + } + + ticker := time.NewTicker(t.duration) + defer ticker.Stop() + + for { + sendPriority() + if priority > t.maxPriority { + return + } + + select { + case <-ticker.C: + case <-t.stop: + return + } + } +} + +func (t *linearPriorityTicker) Stop() { + close(t.stop) + <-t.done +} diff --git a/go/worker/compute/executor/committee/transactions.go b/go/worker/compute/executor/committee/transactions.go index 8d351ff0c6e..7f5ebd1d88f 100644 --- a/go/worker/compute/executor/committee/transactions.go +++ b/go/worker/compute/executor/committee/transactions.go @@ -6,99 +6,43 @@ import ( "time" "github.com/cenkalti/backoff/v4" + "golang.org/x/exp/maps" cmnBackoff "github.com/oasisprotocol/oasis-core/go/common/backoff" "github.com/oasisprotocol/oasis-core/go/common/crypto/hash" - "github.com/oasisprotocol/oasis-core/go/runtime/transaction" "github.com/oasisprotocol/oasis-core/go/runtime/txpool" "github.com/oasisprotocol/oasis-core/go/worker/common/p2p/txsync" ) -// Guarded by n.commonNode.CrossNode. -func (n *Node) resolveBatchLocked(batch *unresolvedBatch, missingState NodeState) (transaction.RawBatch, error) { - n.logger.Debug("attempting to resolve batch", "batch", batch.String()) - - // TODO: Add metrics for how long it takes to receive the complete batch. - if batch.proposal != nil { - n.commonNode.TxPool.PromoteProposedBatch(batch.proposal.Batch) - } - resolvedBatch, err := batch.resolve(n.commonNode.TxPool) - if err != nil { - n.logger.Error("refusing to process bad batch", "err", err) - // TODO: We should indicate failure. - return nil, err - } - if resolvedBatch == nil { - // Some transactions are missing so we cannot start processing the batch just yet. - // Request transactions from peers. - n.logger.Debug("some transactions are missing", "num_missing", len(batch.missingTxs)) - n.transitionLocked(missingState) - - if n.missingTxsCancel != nil { - n.missingTxsCancel() // Cancel any outstanding requests. - } - var ctx context.Context - ctx, n.missingTxsCancel = context.WithCancel(n.roundCtx) - go n.requestMissingTransactions(ctx) - } - return resolvedBatch, nil -} - -// getBatchFromState extracts an unresolved batch from the given node state. -// -// If the state does not contain an unresolved batch, nil is returned. -func getBatchFromState(state NodeState) (batch *unresolvedBatch, isWaitingEv bool) { - switch s := state.(type) { - case StateWaitingForTxs: - batch = s.batch - case StateWaitingForEvent: - batch = s.batch - isWaitingEv = true - default: - } - return -} - func (n *Node) handleNewCheckedTransactions(txs []*txpool.PendingCheckTransaction) { - // Check if we are waiting for new transactions. - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() - - batch, isWaitingEv := getBatchFromState(n.state) - if batch == nil { + state, ok := n.state.(StateWaitingForTxs) + if !ok { return } for _, tx := range txs { - delete(batch.missingTxs, tx.Hash()) + h := tx.Hash() + idx, ok := state.txs[h] + if !ok { + continue + } + delete(state.txs, h) + state.batch[idx] = tx.Raw() + state.bytes += uint64(tx.Size()) } - if len(batch.missingTxs) == 0 { - // We have all transactions, signal the node to start processing the batch. + + if len(state.txs) == 0 { n.logger.Info("received all transactions needed for batch processing") - if !isWaitingEv { - n.startProcessingBatchLocked(batch) - } + } + + // The error will be addressed latter when the state will be updated. + if state.maxBytes > 0 && state.bytes > state.maxBytes { + n.logger.Info("the size of received transactions is too large") } } -func (n *Node) requestMissingTransactions(ctx context.Context) { +func (n *Node) requestMissingTransactions(ctx context.Context, txHashes []hash.Hash) { requestOp := func() error { - // Determine what transactions are missing. - txHashes := func() []hash.Hash { - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() - - batch, _ := getBatchFromState(n.state) - if batch == nil { - return nil - } - - txHashes := make([]hash.Hash, 0, len(batch.missingTxs)) - for txHash := range batch.missingTxs { - txHashes = append(txHashes, txHash) - } - return txHashes - }() if len(txHashes) == 0 { return nil } @@ -127,12 +71,15 @@ func (n *Node) requestMissingTransactions(ctx context.Context) { // Queue all transactions in the transaction pool. n.commonNode.TxPool.SubmitProposedBatch(rsp.Txs) - // Check if there are still missing transactions and perform another request. - if _, missingTxs := n.commonNode.TxPool.GetKnownBatch(txHashes); len(missingTxs) > 0 { - return fmt.Errorf("need to resolve more transactions") + // Check if there are still missing transactions. + _, missingTxs := n.commonNode.TxPool.GetKnownBatch(txHashes) + if len(missingTxs) == 0 { + return nil } + txHashes = maps.Keys(missingTxs) - return nil + // Perform another request. + return fmt.Errorf("need to resolve more transactions") } // Retry until we have resolved all transactions (or round context expires). @@ -146,17 +93,5 @@ func (n *Node) requestMissingTransactions(ctx context.Context) { return } - // We have all transactions, signal the node to start processing the batch. - n.commonNode.CrossNode.Lock() - defer n.commonNode.CrossNode.Unlock() - - batch, isWaitingEv := getBatchFromState(n.state) - if batch == nil { - return - } - n.logger.Info("received all transactions needed for batch processing") - if !isWaitingEv { - n.startProcessingBatchLocked(batch) - } } diff --git a/go/worker/compute/executor/tests/tester.go b/go/worker/compute/executor/tests/tester.go index f19639ac08d..aac6950ef39 100644 --- a/go/worker/compute/executor/tests/tester.go +++ b/go/worker/compute/executor/tests/tester.go @@ -113,9 +113,6 @@ func testQueueTx( // Node should transition to ProcessingBatch state. waitForNodeTransition(t, stateCh, committee.ProcessingBatch) - // Node should transition to WaitingForFinalize state. - waitForNodeTransition(t, stateCh, committee.WaitingForFinalize) - // Node should transition to WaitingForBatch state and a block should be // finalized containing our batch. waitForNodeTransition(t, stateCh, committee.WaitingForBatch) diff --git a/go/worker/keymanager/watcher.go b/go/worker/keymanager/watcher.go index f5e484cf1ac..a1c729e5d90 100644 --- a/go/worker/keymanager/watcher.go +++ b/go/worker/keymanager/watcher.go @@ -73,14 +73,14 @@ func (knw *kmNodeWatcher) watchNodes() { // Rebuild the access policy, something has changed. var nodes []*node.Node - peers := make(map[signature.PublicKey]bool) + peers := make(map[signature.PublicKey]struct{}) for id := range activeNodes { n := watcher.Lookup(id) if n == nil { continue } nodes = append(nodes, n) - peers[n.P2P.ID] = true + peers[n.P2P.ID] = struct{}{} } knw.w.setAccessList(knw.w.runtimeID, nodes) diff --git a/go/worker/storage/committee/node.go b/go/worker/storage/committee/node.go index 69b14361d45..4c0f3a4dcee 100644 --- a/go/worker/storage/committee/node.go +++ b/go/worker/storage/committee/node.go @@ -342,6 +342,11 @@ func (n *Node) HandlePeerTx(ctx context.Context, tx []byte) error { return nil } +// HandleNewConsensusBlockLocked is guarded by CrossNode. +func (n *Node) HandleNewConsensusBlockLocked(*consensus.Block) { + // Nothing to do here. +} + // HandleEpochTransitionLocked is guarded by CrossNode. func (n *Node) HandleEpochTransitionLocked(snapshot *committee.EpochSnapshot) { // Nothing to do here.