Skip to content

Commit

Permalink
keymanager/src/churp: Authorize key share queries
Browse files Browse the repository at this point in the history
  • Loading branch information
peternose committed Jun 28, 2024
1 parent 5296783 commit 81dfcd7
Show file tree
Hide file tree
Showing 15 changed files with 164 additions and 31 deletions.
28 changes: 28 additions & 0 deletions go/consensus/cometbft/apps/keymanager/churp/txs.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import (
"github.com/oasisprotocol/oasis-core/go/common"
"github.com/oasisprotocol/oasis-core/go/common/crypto/signature"
"github.com/oasisprotocol/oasis-core/go/common/node"
"github.com/oasisprotocol/oasis-core/go/common/version"
tmapi "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/api"
churpState "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/apps/keymanager/churp/state"
kmCommon "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/apps/keymanager/common"
Expand All @@ -19,6 +20,11 @@ import (
)

func (ext *churpExt) create(ctx *tmapi.Context, req *churp.CreateRequest) error {
// Make sure the `MayQuery` field is empty until the next breaking upgrade.
if err := verifyPolicy(ctx, &req.Policy); err != nil {
return err
}

// Prepare state.
state := churpState.NewMutableState(ctx.State())

Expand Down Expand Up @@ -125,6 +131,11 @@ func (ext *churpExt) create(ctx *tmapi.Context, req *churp.CreateRequest) error
}

func (ext *churpExt) update(ctx *tmapi.Context, req *churp.UpdateRequest) error {
// Make sure the `MayQuery` field is empty until the next breaking upgrade.
if err := verifyPolicy(ctx, req.Policy); err != nil {
return err
}

// Prepare state.
state := churpState.NewMutableState(ctx.State())

Expand Down Expand Up @@ -545,3 +556,20 @@ func resetHandoff(status *churp.Status, nextHandoff beacon.EpochTime) {
status.NextChecksum = nil
status.Applications = nil
}

func verifyPolicy(ctx *tmapi.Context, policy *churp.SignedPolicySGX) error {
// Allow non-empty `MayQuery` field with the 24.2 release.
regState := registryState.NewMutableState(ctx.State())
regParams, err := regState.ConsensusParameters(ctx)
if err != nil {
return fmt.Errorf("failed to load registry consensus parameters: %w", err)
}
if regParams.SoftwareVersion != nil && regParams.SoftwareVersion.ToU64() >= version.MustFromString("24.2").ToU64() {
return nil
}

if policy != nil && policy.Policy.MayQuery != nil {
return api.ErrInvalidArgument
}
return nil
}
10 changes: 8 additions & 2 deletions go/keymanager/churp/policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ package churp
import (
"fmt"

"github.com/oasisprotocol/oasis-core/go/common"
"github.com/oasisprotocol/oasis-core/go/common/cbor"
"github.com/oasisprotocol/oasis-core/go/common/crypto/signature"
"github.com/oasisprotocol/oasis-core/go/common/sgx"
Expand All @@ -12,20 +13,25 @@ import (
var PolicySGXSignatureContext = signature.NewContext("oasis-core/keymanager/churp: policy")

// PolicySGX represents an SGX access control policy used to authenticate
// key manager enclaves during handoffs.
// key manager enclaves during handoffs and remote client enclaves when
// querying key shares.
type PolicySGX struct {
Identity

// Serial is the monotonically increasing policy serial number.
Serial uint32 `json:"serial"`

// MayShare is the vector of enclave identities from which a share can be
// obtained during handouts.
// obtained during handoffs.
MayShare []sgx.EnclaveIdentity `json:"may_share"`

// MayJoin is the vector of enclave identities that may form the new
// committee in the next handoffs.
MayJoin []sgx.EnclaveIdentity `json:"may_join"`

// MayQuery is the map of runtime identities to the vector of enclave
// identities that may query key shares.
MayQuery map[common.Namespace][]sgx.EnclaveIdentity `json:"may_query,omitempty"`
}

// SanityCheck verifies the validity of the policy.
Expand Down
6 changes: 4 additions & 2 deletions go/keymanager/secrets/policy_sgx.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,10 @@ type EnclavePolicySGX struct {
MayQuery map[common.Namespace][]sgx.EnclaveIdentity `json:"may_query"`

// MayReplicate is the vector of enclave IDs that may retrieve the master
// secret (Note: Each enclave ID may always implicitly replicate from other
// instances of itself).
// secret.
//
// NOTE: Each enclave ID may always implicitly replicate from other
// instances of itself.
MayReplicate []sgx.EnclaveIdentity `json:"may_replicate"`
}

Expand Down
9 changes: 9 additions & 0 deletions go/oasis-node/cmd/genesis/genesis.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/oasisprotocol/oasis-core/go/common/logging"
"github.com/oasisprotocol/oasis-core/go/common/node"
"github.com/oasisprotocol/oasis-core/go/common/quantity"
"github.com/oasisprotocol/oasis-core/go/common/version"
consensus "github.com/oasisprotocol/oasis-core/go/consensus/api"
"github.com/oasisprotocol/oasis-core/go/consensus/api/transaction"
cmt "github.com/oasisprotocol/oasis-core/go/consensus/cometbft/api"
Expand Down Expand Up @@ -65,6 +66,7 @@ const (
CfgRegistryTEEFeaturesSGXSignedAttestations = "registry.tee_features.sgx.signed_attestations"
CfgRegistryTEEFeaturesSGXDefaultMaxAttestationAge = "registry.tee_features.sgx.default_max_attestation_age"
CfgRegistryTEEFeaturesFreshnessProofs = "registry.tee_features.freshness_proofs"
CfgRegistrySoftwareVersion = "registry.software_version"

// Scheduler config flags.
cfgSchedulerMinValidators = "scheduler.min_validators"
Expand Down Expand Up @@ -380,6 +382,12 @@ func AppendRegistryState(doc *genesis.Document, entities, runtimes, nodes []stri
regSt.Parameters.EnableKeyManagerCHURP = true
}

version, err := version.FromString(viper.GetString(CfgRegistrySoftwareVersion))
if err != nil {
return err
}
regSt.Parameters.SoftwareVersion = &version

entMap := make(map[signature.PublicKey]bool)
appendToEntities := func(signedEntity *entity.SignedEntity, ent *entity.Entity) error {
if entMap[ent.ID] {
Expand Down Expand Up @@ -804,6 +812,7 @@ func init() {
initGenesisFlags.Bool(CfgRegistryTEEFeaturesSGXSignedAttestations, true, "enable SGX RAK-signed attestations")
initGenesisFlags.Uint64(CfgRegistryTEEFeaturesSGXDefaultMaxAttestationAge, 1200, "default max attestation age (SGX RAK-signed attestations must be enabled") // ~2 hours at 6 sec per block.
initGenesisFlags.Bool(CfgRegistryTEEFeaturesFreshnessProofs, true, "enable freshness proofs")
initGenesisFlags.String(CfgRegistrySoftwareVersion, "24.0", "latest consensus breaking software version")
_ = initGenesisFlags.MarkHidden(CfgRegistryDebugAllowUnroutableAddresses)
_ = initGenesisFlags.MarkHidden(CfgRegistryDebugAllowTestRuntimes)

Expand Down
1 change: 1 addition & 0 deletions go/oasis-test-runner/oasis/network.go
Original file line number Diff line number Diff line change
Expand Up @@ -795,6 +795,7 @@ func (net *Network) MakeGenesis() error {
"--" + genesis.CfgRegistryEnableRuntimeGovernanceModels, "entity,runtime",
"--" + genesis.CfgRegistryDebugAllowUnroutableAddresses, "true",
"--" + genesis.CfgRegistryDebugAllowTestRuntimes, "true",
"--" + genesis.CfgRegistrySoftwareVersion, "24.2",
"--" + genesis.CfgSchedulerMaxValidatorsPerEntity, strconv.Itoa(len(net.Validators())),
"--" + genesis.CfgConsensusGasCostsTxByte, strconv.FormatUint(uint64(net.cfg.Consensus.Parameters.GasCosts[consensusGenesis.GasOpTxByte]), 10),
"--" + genesis.CfgConsensusStateCheckpointInterval, strconv.FormatUint(net.cfg.Consensus.Parameters.StateCheckpointInterval, 10),
Expand Down
5 changes: 5 additions & 0 deletions go/oasis-test-runner/scenario/e2e/runtime/helpers_churp.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ import (
"fmt"

beacon "github.com/oasisprotocol/oasis-core/go/beacon/api"
"github.com/oasisprotocol/oasis-core/go/common"
"github.com/oasisprotocol/oasis-core/go/common/sgx"
"github.com/oasisprotocol/oasis-core/go/consensus/api/transaction"
"github.com/oasisprotocol/oasis-core/go/keymanager/api"
Expand Down Expand Up @@ -35,6 +36,10 @@ func (sc *Scenario) createChurp(ctx context.Context, id uint8, threshold uint8,
req.Policy.Policy.MayShare = []sgx.EnclaveIdentity{*enclaveID}
}

if enclaveID := sc.Net.Runtimes()[1].GetEnclaveIdentity(0); enclaveID != nil {
req.Policy.Policy.MayQuery = map[common.Namespace][]sgx.EnclaveIdentity{KeyValueRuntimeID: {*enclaveID}}
}

if err := req.Policy.Sign(api.TestSigners); err != nil {
return err
}
Expand Down
3 changes: 3 additions & 0 deletions go/registry/api/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1417,6 +1417,9 @@ type ConsensusParameters struct {

// MaxRuntimeDeployments is the maximum number of runtime deployments.
MaxRuntimeDeployments uint8 `json:"max_runtime_deployments,omitempty"`

// SoftwareVersion represents the latest consensus breaking software version.
SoftwareVersion *version.Version `json:"software_version,omitempty"`
}

// ConsensusParameterChanges are allowed registry consensus parameter changes.
Expand Down
55 changes: 44 additions & 11 deletions go/worker/keymanager/churp.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
cmnBackoff "github.com/oasisprotocol/oasis-core/go/common/backoff"
"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/config"
consensus "github.com/oasisprotocol/oasis-core/go/consensus/api"
"github.com/oasisprotocol/oasis-core/go/keymanager/churp"
Expand Down Expand Up @@ -66,7 +67,7 @@ type churpWorker struct {

kmWorker *Worker

mu sync.Mutex
mu sync.RWMutex
churps map[uint8]*churp.Status // Guarded by mutex.

watcher *nodeWatcher
Expand Down Expand Up @@ -147,12 +148,40 @@ func (w *churpWorker) Authorize(ctx context.Context, method string, kind enclave
}
}

func (w *churpWorker) authorizeNode(_ context.Context, peerID core.PeerID) error {
// TODO: Which nodes are allowed to query key shares?
if w.kmWorker.accessList.Runtimes(peerID).Empty() {
func (w *churpWorker) authorizeNode(ctx context.Context, peerID core.PeerID) error {
rt, err := w.kmWorker.runtime.RegistryDescriptor(ctx)
if err != nil {
return err
}

switch rt.TEEHardware {
case node.TEEHardwareInvalid:
// Insecure key manager enclaves can be queried by all runtimes (used for testing).
return nil
case node.TEEHardwareIntelSGX:
// Secure key manager enclaves can be queried by runtimes specified in the policy.
w.mu.RLock()
statuses := maps.Values(w.churps)
w.mu.RUnlock()

// Retrieve the list of runtimes that the peer participates in.
rts := w.kmWorker.accessList.Runtimes(peerID)

// Grant access if the peer participates in any allowed runtime.
for _, status := range statuses {
if status == nil {
continue
}
for rt := range status.Policy.Policy.MayQuery {
if rts.Contains(rt) {
return nil
}
}
}
return fmt.Errorf("request not allowed")
default:
return fmt.Errorf("unsupported hardware: %s", rt.TEEHardware)
}
return nil
}

func (w *churpWorker) authorizeKeyManager(peerID core.PeerID) error {
Expand All @@ -177,8 +206,8 @@ func (w *churpWorker) Initialized() <-chan struct{} {

// GetStatus returns the worker status.
func (w *churpWorker) GetStatus() workerKm.ChurpStatus {
w.mu.Lock()
defer w.mu.Unlock()
w.mu.RLock()
defer w.mu.RUnlock()

status := workerKm.ChurpStatus{
Schemes: make(map[uint8]workerKm.ChurpSchemeStatus),
Expand Down Expand Up @@ -256,14 +285,16 @@ func (w *churpWorker) handleNewBlock(blk *consensus.Block) {

// handleStatusUpdate is responsible for handling status update.
func (w *churpWorker) handleStatusUpdate(status *churp.Status) {
w.mu.Lock()
defer w.mu.Unlock()

// Skip schemes we are not involved in.
if status.RuntimeID != w.kmWorker.runtimeID {
return
}
if _, ok := w.churps[status.ID]; !ok {

w.mu.RLock()
_, ok := w.churps[status.ID]
w.mu.RUnlock()

if !ok {
return
}

Expand All @@ -272,7 +303,9 @@ func (w *churpWorker) handleStatusUpdate(status *churp.Status) {
)

// Update status.
w.mu.Lock()
w.churps[status.ID] = status
w.mu.Unlock()

// Notify all workers about the new status.
w.watcher.Update(status)
Expand Down
25 changes: 22 additions & 3 deletions keymanager/src/churp/handler.rs
Original file line number Diff line number Diff line change
Expand Up @@ -404,13 +404,13 @@ impl Churp {
/// share needs to be kept secret and generated only for authorized nodes.
pub fn key_share(
&self,
_ctx: &RpcContext,
ctx: &RpcContext,
req: &KeyShareRequest,
) -> Result<EncodedEncryptedPoint> {
// TODO: Add authorization.

let status = self.verify_last_handoff(req.id, req.runtime_id, req.epoch)?;

self.verify_rt_enclave(ctx, &status.policy, &req.key_runtime_id)?;

match status.suite_id {
SuiteId::NistP384Sha3_384 => {
self.make_key_share::<p384::Sha3_384>(&req.key_id.0, &status)
Expand Down Expand Up @@ -1304,6 +1304,25 @@ impl Churp {
Ok(())
}

/// Authorizes the remote runtime enclave so that secret data is never
/// revealed to an unauthorized enclave.
fn verify_rt_enclave(
&self,
ctx: &RpcContext,
policy: &SignedPolicySGX,
runtime_id: &Namespace,
) -> Result<()> {
if Self::ignore_policy() {
return Ok(());
}
let remote_enclave = Self::remote_enclave(ctx)?;
let policy = self.policies.verify(policy)?;
if !policy.may_query(remote_enclave, runtime_id) {
return Err(Error::NotAuthorized.into());
}
Ok(())
}

/// Returns the session RAK of the remote enclave.
fn remote_rak(ctx: &RpcContext) -> Result<PublicKey> {
let si = ctx.session_info.as_ref();
Expand Down
33 changes: 22 additions & 11 deletions keymanager/src/churp/policy.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,14 @@
use std::{
cmp::Ordering,
collections::{HashMap, HashSet},
iter::FromIterator,
sync::{Arc, Mutex},
};

use anyhow::Result;

use oasis_core_runtime::{
common::sgx::EnclaveIdentity,
common::{namespace::Namespace, sgx::EnclaveIdentity},
consensus::keymanager::churp::{PolicySGX, SignedPolicySGX},
};

Expand Down Expand Up @@ -69,9 +70,13 @@ pub struct VerifiedPolicy {
/// during handouts.
pub may_share: HashSet<EnclaveIdentity>,

/// A hash of enclave identities that may form the new committee
/// A set of enclave identities that may form the new committee
/// in the next handoffs.
pub may_join: HashSet<EnclaveIdentity>,

/// A map of runtime identities and their respective sets of enclave
/// identities that are allowed to query key shares.
pub may_query: HashMap<Namespace, HashSet<EnclaveIdentity>>,
}

impl VerifiedPolicy {
Expand All @@ -80,20 +85,17 @@ impl VerifiedPolicy {
/// The provided policy should be valid, signed by trusted signers,
/// and published in the consensus layer state.
fn new(verified_policy: &PolicySGX) -> Result<Self> {
let mut may_share = HashSet::new();
for enclave_identity in &verified_policy.may_share {
may_share.insert(enclave_identity.clone());
}

let mut may_join = HashSet::new();
for enclave_identity in &verified_policy.may_join {
may_join.insert(enclave_identity.clone());
}
let may_share = HashSet::from_iter(verified_policy.may_share.iter().cloned());
let may_join = HashSet::from_iter(verified_policy.may_join.iter().cloned());
let may_query = HashMap::from_iter(verified_policy.may_query.iter().map(
|(runtime_id, enclaves)| (*runtime_id, HashSet::from_iter(enclaves.iter().cloned())),
));

Ok(Self {
serial: verified_policy.serial,
may_share,
may_join,
may_query,
})
}

Expand All @@ -108,4 +110,13 @@ impl VerifiedPolicy {
pub fn may_join(&self, remote_enclave: &EnclaveIdentity) -> bool {
self.may_join.contains(remote_enclave)
}

/// Returns true iff the remote enclave is allowed to query key shares
/// for the given runtime.
pub fn may_query(&self, remote_enclave: &EnclaveIdentity, runtime_id: &Namespace) -> bool {
self.may_query
.get(runtime_id)
.map(|may_query_runtime| may_query_runtime.contains(remote_enclave))
.unwrap_or(false)
}
}
Loading

0 comments on commit 81dfcd7

Please sign in to comment.