From c27ec60d51a8db81e632640fe3e94a22d9501508 Mon Sep 17 00:00:00 2001 From: Jernej Kos Date: Thu, 9 Feb 2023 09:48:12 +0100 Subject: [PATCH] go/runtime: Reduce downtime for TEE runtime upgrades --- .../scenario/e2e/runtime/runtime_upgrade.go | 7 +- go/registry/api/api.go | 5 + go/registry/api/runtime.go | 11 + go/runtime/host/host.go | 4 + go/runtime/host/mock/mock.go | 6 + go/runtime/host/multi/multi.go | 245 ++++++++++++++++-- go/runtime/host/sandbox/sandbox.go | 31 ++- go/runtime/registry/host.go | 23 +- go/worker/common/committee/node.go | 16 +- go/worker/compute/executor/committee/node.go | 20 +- go/worker/keymanager/worker.go | 2 +- 11 files changed, 322 insertions(+), 48 deletions(-) diff --git a/go/oasis-test-runner/scenario/e2e/runtime/runtime_upgrade.go b/go/oasis-test-runner/scenario/e2e/runtime/runtime_upgrade.go index 509261ece8b..d154a002373 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/runtime_upgrade.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/runtime_upgrade.go @@ -198,12 +198,13 @@ func (sc *runtimeUpgradeImpl) Run(childEnv *env.Env) error { if err != nil { return fmt.Errorf("failed to get current epoch: %w", err) } + upgradeEpoch := epoch + 2 // Update runtime to include the new enclave identity. sc.Logger.Info("updating runtime descriptor") newRt := sc.Net.Runtimes()[sc.upgradedRuntimeIndex] newRtDsc := newRt.ToRuntimeDescriptor() - newRtDsc.Deployments[1].ValidFrom = epoch + 1 + newRtDsc.Deployments[1].ValidFrom = upgradeEpoch newTxPath := filepath.Join(childEnv.Dir(), "register_update_compute_runtime.json") if err := cli.Registry.GenerateRegisterRuntimeTx(childEnv.Dir(), newRtDsc, sc.nonce, newTxPath); err != nil { @@ -245,9 +246,9 @@ func (sc *runtimeUpgradeImpl) Run(childEnv *env.Env) error { // Wait for activation epoch. sc.Logger.Info("waiting for runtime upgrade epoch", - "epoch", epoch+1, + "epoch", upgradeEpoch, ) - if err := sc.Net.Controller().Beacon.WaitEpoch(ctx, epoch+1); err != nil { + if err := sc.Net.Controller().Beacon.WaitEpoch(ctx, upgradeEpoch); err != nil { return fmt.Errorf("failed to wait for epoch: %w", err) } diff --git a/go/registry/api/api.go b/go/registry/api/api.go index 08623eb4833..34702a9a0f8 100644 --- a/go/registry/api/api.go +++ b/go/registry/api/api.go @@ -974,6 +974,11 @@ func verifyNodeRuntimeChanges( // can chose not to include it in the registration. continue } + if vi := rtDesc.DeploymentForVersion(version); vi == nil { + // If the missing version is no longer scheduled, it is + // fine if the node does not include it. + continue + } logger.Error("RegisterNode: trying to update runtimes, current version is misssing in new set", "runtime_id", id, diff --git a/go/registry/api/runtime.go b/go/registry/api/runtime.go index e58863860bc..e5ea894bdf5 100644 --- a/go/registry/api/runtime.go +++ b/go/registry/api/runtime.go @@ -510,6 +510,17 @@ func (r *Runtime) ActiveDeployment(now beacon.EpochTime) *VersionInfo { return activeDeployment } +// NextDeployment returns the first deployment that will become active next if it exists. +func (r *Runtime) NextDeployment(now beacon.EpochTime) *VersionInfo { + for _, deployment := range r.Deployments { + // Find the first version that will become the active deployment next. + if deployment.ValidFrom > now { + return deployment + } + } + return nil +} + // DeploymentForVersion returns the deployment corresponding to the passed version if it exists. func (r *Runtime) DeploymentForVersion(v version.Version) *VersionInfo { for _, deployment := range r.Deployments { diff --git a/go/runtime/host/host.go b/go/runtime/host/host.go index 4642f010bdd..71d9d4c3c6e 100644 --- a/go/runtime/host/host.go +++ b/go/runtime/host/host.go @@ -52,6 +52,10 @@ type Runtime interface { // GetInfo retrieves the runtime information. GetInfo(ctx context.Context) (*protocol.RuntimeInfoResponse, error) + // GetCapabilityTEE retrieves the CapabilityTEE of the runtime. It may be nil in case the + // runtime is not running inside a TEE. + GetCapabilityTEE(ctx context.Context) (*node.CapabilityTEE, error) + // Call sends a request message to the runtime over the Runtime Host Protocol and waits for the // response (which may be a failure). Call(ctx context.Context, body *protocol.Body) (*protocol.Body, error) diff --git a/go/runtime/host/mock/mock.go b/go/runtime/host/mock/mock.go index 34621f44b53..218ba459d3f 100644 --- a/go/runtime/host/mock/mock.go +++ b/go/runtime/host/mock/mock.go @@ -10,6 +10,7 @@ import ( "github.com/oasisprotocol/oasis-core/go/common/cbor" "github.com/oasisprotocol/oasis-core/go/common/crypto/hash" "github.com/oasisprotocol/oasis-core/go/common/errors" + "github.com/oasisprotocol/oasis-core/go/common/node" "github.com/oasisprotocol/oasis-core/go/common/pubsub" "github.com/oasisprotocol/oasis-core/go/common/version" "github.com/oasisprotocol/oasis-core/go/roothash/api/commitment" @@ -57,6 +58,11 @@ func (r *runtime) GetInfo(ctx context.Context) (rsp *protocol.RuntimeInfoRespons }, nil } +// Implements host.Runtime. +func (r *runtime) GetCapabilityTEE(ctx context.Context) (rsp *node.CapabilityTEE, err error) { + return nil, nil +} + // Implements host.Runtime. func (r *runtime) Call(ctx context.Context, body *protocol.Body) (*protocol.Body, error) { switch { diff --git a/go/runtime/host/multi/multi.go b/go/runtime/host/multi/multi.go index e77b240a329..4d732a5c59b 100644 --- a/go/runtime/host/multi/multi.go +++ b/go/runtime/host/multi/multi.go @@ -10,10 +10,12 @@ import ( "sync" "github.com/cenkalti/backoff/v4" + "github.com/hashicorp/go-multierror" "github.com/oasisprotocol/oasis-core/go/common" cmnBackoff "github.com/oasisprotocol/oasis-core/go/common/backoff" "github.com/oasisprotocol/oasis-core/go/common/logging" + "github.com/oasisprotocol/oasis-core/go/common/node" "github.com/oasisprotocol/oasis-core/go/common/pubsub" "github.com/oasisprotocol/oasis-core/go/common/version" "github.com/oasisprotocol/oasis-core/go/runtime/host" @@ -31,14 +33,57 @@ var ( type aggregatedHost struct { host host.Runtime - ch <-chan *host.Event - sub pubsub.ClosableSubscription - stopCh chan struct{} - stoppedCh chan struct{} + ch <-chan *host.Event + sub pubsub.ClosableSubscription + stopCh chan struct{} + stoppedCh chan struct{} + stopDiscardCh chan struct{} + stoppedDiscardCh chan *host.Event version version.Version } +func (ah *aggregatedHost) startDiscard() { + go func() { + defer close(ah.stoppedDiscardCh) + + var startedEv *host.Event + defer func() { + // In case we observed a started event, forward it. + ah.stoppedDiscardCh <- startedEv + }() + + for { + select { + case <-ah.stopDiscardCh: + return + case ev := <-ah.ch: + // Store the last started event, clearing out if any stopped event received. + if ev.Started != nil { + startedEv = ev + } + if ev.Stopped != nil { + startedEv = nil + } + // Make sure the started event's CapabilityTEE is always the latest one. + if ev.Updated != nil && startedEv != nil { + startedEv.Started.CapabilityTEE = ev.Updated.CapabilityTEE + } + } + } + }() +} + +func (ah *aggregatedHost) stopDiscard(agg *Aggregate) { + close(ah.stopDiscardCh) + ev := <-ah.stoppedDiscardCh + + // Propagate captured started event. + if ev != nil && agg != nil { + agg.notifier.Broadcast(ev) + } +} + func (ah *aggregatedHost) startPassthrough(agg *Aggregate) { go func() { defer close(ah.stoppedCh) @@ -68,6 +113,7 @@ type Aggregate struct { hosts map[version.Version]*aggregatedHost active *aggregatedHost + next *aggregatedHost notifier *pubsub.Broker } @@ -88,6 +134,17 @@ func (agg *Aggregate) GetInfo(ctx context.Context) (*protocol.RuntimeInfoRespons return agg.active.host.GetInfo(ctx) } +// GetCapabilityTEE implements host.Runtime. +func (agg *Aggregate) GetCapabilityTEE(ctx context.Context) (*node.CapabilityTEE, error) { + agg.l.RLock() + defer agg.l.RUnlock() + + if agg.active == nil { + return nil, ErrNoActiveVersion + } + return agg.active.host.GetCapabilityTEE(ctx) +} + // Call implements host.Runtime. func (agg *Aggregate) Call(ctx context.Context, body *protocol.Body) (rsp *protocol.Body, err error) { callFn := func() error { @@ -121,10 +178,18 @@ func (agg *Aggregate) UpdateCapabilityTEE(ctx context.Context) error { agg.l.RLock() defer agg.l.RUnlock() - if agg.active == nil { - return ErrNoActiveVersion + var errs error + if agg.active != nil { + if err := agg.active.host.UpdateCapabilityTEE(ctx); err != nil { + errs = multierror.Append(errs, err) + } + } + if agg.next != nil { + if err := agg.next.host.UpdateCapabilityTEE(ctx); err != nil { + errs = multierror.Append(errs, err) + } } - return agg.active.host.UpdateCapabilityTEE(ctx) + return errs } // WatchEvents implements host.Runtime. @@ -169,21 +234,53 @@ func (agg *Aggregate) Stop() { // this leaves the notifier lying around. agg.stopActiveLocked() + agg.stopNextLocked() } -// SetVersion sets the active runtime version. This routine will: +// GetVersion retrieves the runtime host for the specified version. +func (agg *Aggregate) GetVersion(ctx context.Context, version version.Version) (host.Runtime, error) { + agg.l.RLock() + defer agg.l.RUnlock() + + host := agg.hosts[version] + if host == nil { + return nil, ErrNoSuchVersion + } + // Only allow fetching either the active or next versions. + if host != agg.active && host != agg.next { + return nil, ErrNoSuchVersion + } + return host.host, nil +} + +// SetVersion sets the active and next runtime versions. This routine will: // - Do nothing if the active version is already the requested version. // - Unconditionally tear down the currently active version (via Stop()). // - Start the newly active version if it exists. -func (agg *Aggregate) SetVersion(ctx context.Context, version version.Version) error { +// - Do nothing if the next version is already the requested version. +// - Start the next version if it exists. +func (agg *Aggregate) SetVersion(ctx context.Context, active version.Version, next *version.Version) error { agg.l.Lock() defer agg.l.Unlock() - agg.logger.Info("SetVersion", + agg.logger.Info("set version", "id", agg.ID(), - "version", version, + "active", active, + "next", next, ) + if err := agg.setActiveVersionLocked(ctx, active); err != nil { + return err + } + if err := agg.setNextVersionLocked(ctx, next); err != nil { + return err + } + return nil +} + +func (agg *Aggregate) setActiveVersionLocked(ctx context.Context, version version.Version) error { + // Contract: agg.l already locked for write. + // Ensure that we know about the new version. if agg.hosts[version] == nil { agg.logger.Error("SetVersion: unknown version", @@ -208,18 +305,24 @@ func (agg *Aggregate) SetVersion(ctx context.Context, version version.Version) e } // Get ready to spin up the new runtime. - var err error ah := agg.hosts[version] - host := ah.host - if err = host.Start(); err != nil { - // Do not bail, this can't actually fail in practice because - // the part that does all the work is async. Log something. - agg.logger.Error("SetVersion: failed to start sub-host", - "err", err, - "id", agg.ID(), - "version", version, - ) + if agg.next == ah { + // This is the next active version which has already been started in advance. Clear out the + // currently active next version and stop discarding events. + agg.next = nil + ah.stopDiscard(agg) // Forward any captured started events. + } else { + host := ah.host + if err := host.Start(); err != nil { + // Do not bail, this can't actually fail in practice because + // the part that does all the work is async. Log something. + agg.logger.Error("SetVersion: failed to start sub-host", + "err", err, + "id", agg.ID(), + "version", version, + ) + } } // Assume that the caller is ok with SetVersion acting as a Stop+Start @@ -275,6 +378,92 @@ func (agg *Aggregate) stopActiveLocked() { agg.active = nil } +func (agg *Aggregate) setNextVersionLocked(ctx context.Context, maybeVersion *version.Version) error { + // Contract: agg.l already locked for write. + + // The next version could become unscheduled, in this case tear it down. + if maybeVersion == nil { + agg.stopNextLocked() + return nil + } + version := *maybeVersion + + // Ensure that we know about the next version. + if agg.hosts[version] == nil { + agg.logger.Warn("unsupported next version", + "id", agg.ID(), + "version", version, + ) + // Active version must be unaffected. + return ErrNoSuchVersion + } + + // Ensure next version is not the same as the active version. + if agg.active != nil && agg.hosts[version] == agg.active { + return nil + } + + // If there already is a next version... + if agg.next != nil { + // If it is the same as the requested one, we are done. + if agg.hosts[version] == agg.next { + return nil + } + + // Warn in case the next version is changed but the previous one was not activated yet. + agg.logger.Warn("overwriting next version without activation", + "id", agg.ID(), + "version", version, + "previous_version", agg.next.version, + ) + agg.stopNextLocked() + } + + // Start the next version. + ah := agg.hosts[version] + if err := ah.host.Start(); err != nil { + // Do not bail, this can't actually fail in practice because + // the part that does all the work is async. Log something. + agg.logger.Error("failed to start next version sub-host", + "err", err, + "id", agg.ID(), + "version", version, + ) + } + + // Start discarding events. + ah.startDiscard() + + // Update the next version. + agg.next = ah + + return nil +} + +func (agg *Aggregate) stopNextLocked() { + // Contract: agg.l already locked for write. + + if agg.next == nil { + return + } + + agg.logger.Debug("stopNextLocked", + "id", agg.ID(), + "version", agg.next.version, + ) + + ah := agg.next + ah.stopDiscard(nil) // Drop any captured started events. + + // Terminate the next instance. + ah.host.Stop() + + // Close off the subscription, invalidate the old sub-host. + ah.sub.Close() + agg.hosts[ah.version] = nil + agg.next = nil +} + // New returns a new aggregated runtime. The runtimes provided must be // freshly provisioned (ie: Start() must not have been called). func New( @@ -310,12 +499,14 @@ func New( } ah := &aggregatedHost{ - host: rts[version], - ch: ch, - sub: sub, - stopCh: make(chan struct{}), - stoppedCh: make(chan struct{}), - version: version, + host: rts[version], + ch: ch, + sub: sub, + stopCh: make(chan struct{}), + stoppedCh: make(chan struct{}), + stopDiscardCh: make(chan struct{}), + stoppedDiscardCh: make(chan *host.Event), + version: version, } agg.hosts[version] = ah } diff --git a/go/runtime/host/sandbox/sandbox.go b/go/runtime/host/sandbox/sandbox.go index 9c5e93cf6e7..efd54da732d 100644 --- a/go/runtime/host/sandbox/sandbox.go +++ b/go/runtime/host/sandbox/sandbox.go @@ -3,6 +3,7 @@ package sandbox import ( "context" + "errors" "fmt" "net" "os" @@ -16,6 +17,7 @@ import ( "github.com/oasisprotocol/oasis-core/go/common" cmnBackoff "github.com/oasisprotocol/oasis-core/go/common/backoff" "github.com/oasisprotocol/oasis-core/go/common/logging" + "github.com/oasisprotocol/oasis-core/go/common/node" "github.com/oasisprotocol/oasis-core/go/common/pubsub" "github.com/oasisprotocol/oasis-core/go/common/version" "github.com/oasisprotocol/oasis-core/go/runtime/host" @@ -23,6 +25,8 @@ import ( "github.com/oasisprotocol/oasis-core/go/runtime/host/sandbox/process" ) +var errRuntimeNotReady = errors.New("runtime is not yet ready") + const ( runtimeConnectTimeout = 5 * time.Second runtimeInitTimeout = 1 * time.Second @@ -116,6 +120,7 @@ type sandboxedRuntime struct { notifier *pubsub.Broker notifyUpdateCapabilityTEE *channels.RingChannel + capabilityTEE *node.CapabilityTEE logger *logging.Logger } @@ -132,7 +137,7 @@ func (r *sandboxedRuntime) GetInfo(ctx context.Context) (rsp *protocol.RuntimeIn defer r.RUnlock() if r.conn == nil { - return fmt.Errorf("runtime is not ready") + return errRuntimeNotReady } rsp, err = r.conn.GetInfo(ctx) return err @@ -143,6 +148,17 @@ func (r *sandboxedRuntime) GetInfo(ctx context.Context) (rsp *protocol.RuntimeIn return } +// Implements host.Runtime. +func (r *sandboxedRuntime) GetCapabilityTEE(ctx context.Context) (*node.CapabilityTEE, error) { + r.RLock() + defer r.RUnlock() + + if r.conn == nil { + return nil, errRuntimeNotReady + } + return r.capabilityTEE, nil +} + // Implements host.Runtime. func (r *sandboxedRuntime) Call(ctx context.Context, body *protocol.Body) (rsp *protocol.Body, err error) { callFn := func() error { @@ -150,7 +166,7 @@ func (r *sandboxedRuntime) Call(ctx context.Context, body *protocol.Body) (rsp * defer r.RUnlock() if r.conn == nil { - return fmt.Errorf("runtime is not ready") + return errRuntimeNotReady } rsp, err = r.conn.Call(ctx, body) if err != nil { @@ -225,6 +241,13 @@ func (r *sandboxedRuntime) Stop() { // Implements host.EmitEvent. func (r *sandboxedRuntime) EmitEvent(ev *host.Event) { + // Update runtime's CapabilityTEE in case this is an update event. + if ue := ev.Updated; ue != nil { + r.Lock() + r.capabilityTEE = ue.CapabilityTEE + r.Unlock() + } + r.notifier.Broadcast(ev) } @@ -402,6 +425,7 @@ func (r *sandboxedRuntime) startProcess() (err error) { ok = true r.process = p r.conn = pc + r.capabilityTEE = ev.CapabilityTEE // Notify subscribers that a runtime has been started. r.notifier.Broadcast(&host.Event{Started: ev}) @@ -444,6 +468,7 @@ func (r *sandboxedRuntime) handleAbortRequest(rq *abortRequest) error { r.conn.Close() r.process = nil r.conn = nil + r.capabilityTEE = nil r.Unlock() // Notify subscribers that the runtime has stopped. @@ -470,6 +495,7 @@ func (r *sandboxedRuntime) manager() { r.Lock() r.conn = nil + r.capabilityTEE = nil r.Unlock() } @@ -548,6 +574,7 @@ func (r *sandboxedRuntime) manager() { r.conn.Close() r.process = nil r.conn = nil + r.capabilityTEE = nil r.Unlock() // Notify subscribers that the runtime has stopped. diff --git a/go/runtime/registry/host.go b/go/runtime/registry/host.go index 7736b764279..5cd7a25bba8 100644 --- a/go/runtime/registry/host.go +++ b/go/runtime/registry/host.go @@ -117,8 +117,25 @@ func (n *RuntimeHostNode) WaitHostedRuntime(ctx context.Context) (host.RichRunti return n.GetHostedRuntime(), nil } -// SetHostedRuntimeVersion sets the currently active version for the hosted runtime. -func (n *RuntimeHostNode) SetHostedRuntimeVersion(ctx context.Context, version version.Version) error { +// GetHostedRuntimeCapabilityTEE returns the CapabilityTEE for a specific runtime version. +func (n *RuntimeHostNode) GetHostedRuntimeCapabilityTEE(ctx context.Context, version version.Version) (*node.CapabilityTEE, error) { + n.Lock() + agg := n.agg + n.Unlock() + + if agg == nil { + return nil, fmt.Errorf("runtime not available") + } + + rt, err := agg.GetVersion(ctx, version) + if err != nil { + return nil, err + } + return rt.GetCapabilityTEE(ctx) +} + +// SetHostedRuntimeVersion sets the currently active and next versions for the hosted runtime. +func (n *RuntimeHostNode) SetHostedRuntimeVersion(ctx context.Context, active version.Version, next *version.Version) error { n.Lock() agg := n.agg n.Unlock() @@ -127,7 +144,7 @@ func (n *RuntimeHostNode) SetHostedRuntimeVersion(ctx context.Context, version v return fmt.Errorf("runtime not available") } - return agg.SetVersion(ctx, version) + return agg.SetVersion(ctx, active, next) } // RuntimeHostHandlerFactory is an interface that can be used to create new runtime handlers and diff --git a/go/worker/common/committee/node.go b/go/worker/common/committee/node.go index 59e23eebe0b..54f2ab1d463 100644 --- a/go/worker/common/committee/node.go +++ b/go/worker/common/committee/node.go @@ -13,6 +13,7 @@ import ( "github.com/oasisprotocol/oasis-core/go/common/identity" "github.com/oasisprotocol/oasis-core/go/common/logging" "github.com/oasisprotocol/oasis-core/go/common/version" + "github.com/oasisprotocol/oasis-core/go/config" consensus "github.com/oasisprotocol/oasis-core/go/consensus/api" control "github.com/oasisprotocol/oasis-core/go/control/api" keymanager "github.com/oasisprotocol/oasis-core/go/keymanager/api" @@ -432,10 +433,21 @@ func (n *Node) updateHostedRuntimeVersionLocked() { activeVersion = activeDeploy.Version } - if err := n.SetHostedRuntimeVersion(n.ctx, activeVersion); err != nil { - n.logger.Error("failed to activate runtime version", + // For compute nodes, determine if there is a next version and activate it early. + var nextVersion *version.Version + if config.GlobalConfig.Mode == config.ModeCompute { + nextDeploy := n.CurrentDescriptor.NextDeployment(n.CurrentEpoch) + if nextDeploy != nil { + nextVersion = new(version.Version) + *nextVersion = nextDeploy.Version + } + } + + if err := n.SetHostedRuntimeVersion(n.ctx, activeVersion, nextVersion); err != nil { + n.logger.Error("failed to activate runtime version(s)", "err", err, "version", activeVersion, + "next_version", nextVersion, ) // This is not fatal and it should result in the node declaring itself unavailable. } diff --git a/go/worker/compute/executor/committee/node.go b/go/worker/compute/executor/committee/node.go index e30afac59ba..070e46e1818 100644 --- a/go/worker/compute/executor/committee/node.go +++ b/go/worker/compute/executor/committee/node.go @@ -107,7 +107,6 @@ var ( type Node struct { // nolint: maligned runtimeReady bool runtimeVersion version.Version - runtimeCapabilityTEE *node.CapabilityTEE runtimeTrustSynced bool runtimeTrustSyncCncl context.CancelFunc @@ -1338,20 +1337,23 @@ func (n *Node) nudgeAvailabilityLocked(force bool) { n.roleProvider.SetAvailable(func(nd *node.Node) error { for _, version := range n.commonNode.Runtime.HostVersions() { - // For TEE-enabled runtimes we can only advertise the active version as this will - // otherwise cause additional downtime on upgrades due to capability updates not - // being allowed. - if n.runtimeCapabilityTEE != nil && version != n.runtimeVersion { + // Skip sending any old versions that will never be active again. + if version.ToU64() < n.runtimeVersion.ToU64() { continue } - // Skip sending any old versions that will never be active again. - if version.ToU64() < n.runtimeVersion.ToU64() { + // Obtain CapabilityTEE for the given rutnime version. + capabilityTEE, err := n.commonNode.GetHostedRuntimeCapabilityTEE(n.ctx, version) + if err != nil { + n.logger.Warn("failed to get CapabilityTEE for hosted runtime, skipping", + "err", err, + "version", version, + ) continue } rt := nd.AddOrUpdateRuntime(n.commonNode.Runtime.ID(), version) - rt.Capabilities.TEE = n.runtimeCapabilityTEE + rt.Capabilities.TEE = capabilityTEE } return nil }) @@ -1396,11 +1398,9 @@ func (n *Node) HandleRuntimeHostEventLocked(ev *host.Event) { // We are now able to service requests for this runtime. n.runtimeReady = true n.runtimeVersion = ev.Started.Version - n.runtimeCapabilityTEE = ev.Started.CapabilityTEE case ev.Updated != nil: // Update runtime capabilities. n.runtimeReady = true - n.runtimeCapabilityTEE = ev.Updated.CapabilityTEE case ev.FailedToStart != nil, ev.Stopped != nil: // Runtime failed to start or was stopped -- we can no longer service requests. n.runtimeReady = false diff --git a/go/worker/keymanager/worker.go b/go/worker/keymanager/worker.go index 8010fd24417..37988bcccb4 100644 --- a/go/worker/keymanager/worker.go +++ b/go/worker/keymanager/worker.go @@ -961,7 +961,7 @@ func (w *Worker) worker() { // nolint: gocyclo // Key managers always need to use the enclave version given to them in the bundle // as they need to make sure that replication is possible during upgrades. activeVersion := w.runtime.HostVersions()[0] // Init made sure we have exactly one. - if err = w.SetHostedRuntimeVersion(w.ctx, activeVersion); err != nil { + if err = w.SetHostedRuntimeVersion(w.ctx, activeVersion, nil); err != nil { w.logger.Error("failed to activate runtime version", "err", err, "version", activeVersion,