Skip to content

Commit

Permalink
feat(CON-1419): Deliver VetKeys to execution (#3860)
Browse files Browse the repository at this point in the history
This PR adds a function, which returns the `MasterPublicKeys` and
`NiDkgId` corresponding to the `MasterPublicKeyId`s that are active
VetKD keys in the subnet.

The `MasterPublicKey`s are then delivered to execution. This enabled the
`vetkd_public_key` function to work. The corresponding system test is
adapted to demonstrate that.

The `NiDkgId`s are not delivered to execution yet (this is captured in
CON-1420).
Also, currently there is no unit test to cover the functionality.

---------

Co-authored-by: Leo Eichhorn <99166915+eichhorl@users.noreply.github.com>
  • Loading branch information
Sawchord and eichhorl authored Feb 20, 2025
1 parent c9a3f87 commit 6e32786
Show file tree
Hide file tree
Showing 10 changed files with 200 additions and 83 deletions.
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

3 changes: 2 additions & 1 deletion rs/consensus/dkg/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -7,13 +7,14 @@ description.workspace = true
documentation.workspace = true

[dependencies]

ic-metrics = { path = "../../monitoring/metrics" }
ic-consensus-utils = { path = "../utils" }
ic-interfaces = { path = "../../interfaces" }
ic-interfaces-registry = { path = "../../interfaces/registry" }
ic-interfaces-state-manager = { path = "../../interfaces/state_manager" }
ic-logger = { path = "../../monitoring/logger" }
ic-management-canister-types-private = { path = "../../types/management_canister_types" }
ic-metrics = { path = "../../monitoring/metrics" }
ic-protobuf = { path = "../../protobuf" }
ic-registry-client-helpers = { path = "../../registry/helpers" }
ic-replicated-state = { path = "../../replicated_state" }
Expand Down
5 changes: 4 additions & 1 deletion rs/consensus/dkg/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,10 @@ pub mod dkg_key_manager;
pub mod payload_builder;
pub mod payload_validator;

pub use crate::payload_validator::{DkgPayloadValidationFailure, InvalidDkgPayloadReason};
pub use crate::{
payload_validator::{DkgPayloadValidationFailure, InvalidDkgPayloadReason},
utils::get_vetkey_public_keys,
};

#[cfg(test)]
mod test_utils;
Expand Down
76 changes: 74 additions & 2 deletions rs/consensus/dkg/src/utils.rs
Original file line number Diff line number Diff line change
@@ -1,15 +1,85 @@
use crate::PayloadCreationError;
use ic_consensus_utils::pool_reader::PoolReader;
use ic_interfaces_registry::RegistryClient;
use ic_logger::{warn, ReplicaLogger};
use ic_management_canister_types_private::MasterPublicKeyId;
use ic_registry_client_helpers::subnet::SubnetRegistry;
use ic_types::{
consensus::Block,
crypto::threshold_sig::ni_dkg::{NiDkgDealing, NiDkgId, NiDkgMasterPublicKeyId, NiDkgTag},
consensus::{dkg::Summary, Block},
crypto::{
canister_threshold_sig::MasterPublicKey,
threshold_sig::{
ni_dkg::{NiDkgDealing, NiDkgId, NiDkgMasterPublicKeyId, NiDkgTag},
ThresholdSigPublicKey,
},
AlgorithmId,
},
NodeId, RegistryVersion, SubnetId,
};
use std::collections::{BTreeMap, HashSet};

/// Returns the [`MasterPublicKey`]s and also the [`NiDkgId`]s of the `next_transcript`
/// (or, if unavialable, of the `current_transcript`) corresponding to the [`MasterPublicKeyId`]s
/// active on the subnet.
#[allow(clippy::type_complexity)]
pub fn get_vetkey_public_keys(
summary: &Summary,
logger: &ReplicaLogger,
) -> (
BTreeMap<MasterPublicKeyId, MasterPublicKey>,
BTreeMap<MasterPublicKeyId, NiDkgId>,
) {
// Get all next transcripts
// If there is a current transcript, but no next transcript, use that one instead.
let mut transcripts = summary
.next_transcripts()
.iter()
.collect::<BTreeMap<_, _>>();
for (tag, transcript) in summary.current_transcripts().iter() {
if !transcripts.contains_key(tag) {
warn!(logger, "Reusing current transcript for tag {:?}", tag);
transcripts.insert(tag, transcript);
}
}

let (master_keys, dkg_ids) = transcripts
.iter()
// Filter out transcripts that are not for VetKD
.filter_map(|(tag, &transcript)| match tag {
NiDkgTag::HighThresholdForKey(key_id) => Some((key_id, transcript)),
_ => None,
})
// Try to build the public key from the transcript
.filter_map(
|(key_id, transcript)| match ThresholdSigPublicKey::try_from(transcript) {
Err(err) => {
warn!(
logger,
"Failed to get public key for key id {}: {:?}", key_id, err
);
None
}
Ok(pubkey) => Some((key_id, pubkey, transcript.dkg_id.clone())),
},
)
// Unzip the data into the two maps that delivery needs
.map(|(key_id, pubkey, ni_dkg_id)| {
(
(
key_id.clone().into(),
MasterPublicKey {
algorithm_id: AlgorithmId::ThresBls12_381,
public_key: pubkey.into_bytes().to_vec(),
},
),
(key_id.clone().into(), ni_dkg_id),
)
})
.unzip();

(master_keys, dkg_ids)
}

pub(super) fn get_dealers_from_chain(
pool_reader: &PoolReader<'_>,
block: &Block,
Expand Down Expand Up @@ -162,4 +232,6 @@ mod tests {
matches!(&vetkeys[1], NiDkgMasterPublicKeyId::VetKd(key) if key.name == "second_vet_kd_key")
);
}

// TODO: Unit test for `get_vetkey_public_keys`. (Currently its covered through a system test)
}
48 changes: 23 additions & 25 deletions rs/consensus/src/consensus/batch_delivery.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ use crate::{
},
idkg::utils::{get_idkg_subnet_public_keys, get_pre_signature_ids_to_deliver},
};
use ic_consensus_dkg::get_vetkey_public_keys;
use ic_consensus_utils::{
crypto_hashable_to_seed, membership::Membership, pool_reader::PoolReader,
};
Expand Down Expand Up @@ -135,20 +136,29 @@ pub fn deliver_batches(

let randomness = Randomness::from(crypto_hashable_to_seed(&tape));

// TODO(CON-1419): Add vetKD keys to this map as well
let chain_key_subnet_public_keys = match get_idkg_subnet_public_keys(&block, pool, log) {
Ok(keys) => keys,
Err(e) => {
// Do not deliver batch if we can't find a previous summary block,
// this means we should continue with the latest CUP.
warn!(
every_n_seconds => 5,
log,
"Do not deliver height {:?}: {}", height, e
);
return Ok(last_delivered_batch_height);
}
// Retrieve the dkg summary block
let Some(summary_block) = pool.dkg_summary_block_for_finalized_height(height) else {
warn!(
every_n_seconds => 30,
log,
"Do not deliver height {} because no summary block was found. \
Finalized height: {}",
height,
finalized_height
);
break;
};
let dkg_summary = &summary_block.payload.as_ref().as_summary().dkg;

let mut chain_key_subnet_public_keys = BTreeMap::new();
let mut idkg_subnet_public_keys =
get_idkg_subnet_public_keys(&block, &summary_block, pool, log);
chain_key_subnet_public_keys.append(&mut idkg_subnet_public_keys);

// Add vetKD keys to this map as well
// TODO(CON-1420: Deliver the ni_dkg_ids to the batch as well)
let (mut nidkg_subnet_public_keys, _ni_dkg_ids) = get_vetkey_public_keys(dkg_summary, log);
chain_key_subnet_public_keys.append(&mut nidkg_subnet_public_keys);

let mut batch_stats = BatchStats::new(height);

Expand Down Expand Up @@ -205,18 +215,6 @@ pub fn deliver_batches(
failed_blockmakers: blockmaker_ranking[0..(block.rank.0 as usize)].to_vec(),
};

let Some(summary_block) = pool.dkg_summary_block_for_finalized_height(height) else {
warn!(
every_n_seconds => 30,
log,
"Do not deliver height {} because no summary block was found. \
Finalized height: {}",
height,
finalized_height
);
break;
};
let dkg_summary = &summary_block.payload.as_ref().as_summary().dkg;
let next_checkpoint_height = dkg_summary.get_next_start_height();
let current_interval_length = dkg_summary.interval_length;
let batch = Batch {
Expand Down
28 changes: 11 additions & 17 deletions rs/consensus/src/idkg/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -532,31 +532,25 @@ pub(crate) fn get_pre_signature_ids_to_deliver(
}

/// This function returns the subnet master public keys to be added to the batch, if required.
/// We return `Ok(Some(key))`, if
/// We return the keys, if
/// - The block contains an IDKG payload with current key transcript ref, and
/// - the corresponding transcript exists in past blocks, and
/// - we can extract the threshold master public key from the transcript.
///
/// Otherwise `Ok(None)` is returned.
/// Additionally, we return `Err(string)` if we were unable to find a dkg summary block for the height
/// of the given block (as the lower bound for past blocks to lookup the transcript in). In that case
/// a newer CUP is already present in the pool and we should continue from there.
/// Otherwise no keys are returned.
pub(crate) fn get_idkg_subnet_public_keys(
block: &Block,
current_block: &Block,
last_dkg_summary_block: &Block,
pool: &PoolReader<'_>,
log: &ReplicaLogger,
) -> Result<BTreeMap<MasterPublicKeyId, MasterPublicKey>, String> {
let Some(idkg_payload) = block.payload.as_ref().as_idkg() else {
return Ok(BTreeMap::new());
) -> BTreeMap<MasterPublicKeyId, MasterPublicKey> {
let Some(idkg_payload) = current_block.payload.as_ref().as_idkg() else {
return BTreeMap::new();
};

let Some(summary) = pool.dkg_summary_block_for_finalized_height(block.height) else {
return Err(format!(
"Failed to find dkg summary block for height {}",
block.height
));
};
let chain = pool.pool().build_block_chain(&summary, block);
let chain = pool
.pool()
.build_block_chain(last_dkg_summary_block, current_block);
let block_reader = IDkgBlockReaderImpl::new(chain);

let mut public_keys = BTreeMap::new();
Expand Down Expand Up @@ -587,7 +581,7 @@ pub(crate) fn get_idkg_subnet_public_keys(
}
}

Ok(public_keys)
public_keys
}

fn get_subnet_master_public_key(
Expand Down
55 changes: 53 additions & 2 deletions rs/tests/consensus/tecdsa/utils/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ use ic_management_canister_types_private::{
DerivationPath, ECDSAPublicKeyArgs, ECDSAPublicKeyResponse, EcdsaCurve, EcdsaKeyId,
MasterPublicKeyId, Payload, SchnorrAlgorithm, SchnorrKeyId, SchnorrPublicKeyArgs,
SchnorrPublicKeyResponse, SignWithECDSAArgs, SignWithECDSAReply, SignWithSchnorrArgs,
SignWithSchnorrReply,
SignWithSchnorrReply, VetKdKeyId, VetKdPublicKeyArgs, VetKdPublicKeyResult,
};
use ic_message::ForwardParams;
use ic_nervous_system_common_test_keys::{TEST_NEURON_1_ID, TEST_NEURON_1_OWNER_KEYPAIR};
Expand Down Expand Up @@ -298,7 +298,9 @@ pub async fn get_public_key_with_retries(
MasterPublicKeyId::Schnorr(key_id) => {
get_schnorr_public_key_with_retries(key_id, msg_can, logger, retries).await
}
MasterPublicKeyId::VetKd(_) => panic!("not applicable to vetKD"),
MasterPublicKeyId::VetKd(key_id) => {
get_vetkd_public_key_with_retries(key_id, msg_can, logger, retries).await
}
}
}

Expand Down Expand Up @@ -419,6 +421,55 @@ pub async fn get_schnorr_public_key_with_retries(
Ok(public_key)
}

pub async fn get_vetkd_public_key_with_retries(
key_id: &VetKdKeyId,
msg_can: &MessageCanister<'_>,
logger: &Logger,
retries: u64,
) -> Result<Vec<u8>, AgentError> {
let public_key_request = VetKdPublicKeyArgs {
canister_id: None,
derivation_path: DerivationPath::new(vec![]),
key_id: key_id.clone(),
};
info!(
logger,
"Sending a 'get vetkd public key' request: {:?}", public_key_request
);

let mut count = 0;
let public_key = loop {
let res = msg_can
.forward_to(
&Principal::management_canister(),
"vetkd_public_key",
Encode!(&public_key_request).unwrap(),
)
.await;
match res {
Ok(bytes) => {
let key = VetKdPublicKeyResult::decode(&bytes)
.expect("failed to decode VetKdPublicKeyResult");
break key.public_key;
}
Err(err) => {
count += 1;
if count < retries {
debug!(
logger,
"vetkd_public_key returns `{}`. Trying again in 2 seconds...", err
);
tokio::time::sleep(Duration::from_secs(2)).await;
} else {
return Err(err);
}
}
}
};

Ok(public_key)
}

pub async fn get_public_key_with_logger(
key_id: &MasterPublicKeyId,
msg_can: &MessageCanister<'_>,
Expand Down
1 change: 1 addition & 0 deletions rs/tests/consensus/vetkd/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ system_test_nns(
"//rs/types/types",
"@crate_index//:anyhow",
"@crate_index//:ic-agent",
"@crate_index//:ic_bls12_381",
"@crate_index//:slog",
],
)
1 change: 1 addition & 0 deletions rs/tests/consensus/vetkd/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ ic-system-test-driver = { path = "../../driver" }
ic-types = { path = "../../../types/types" }
ic_consensus_system_test_utils = { path = "../utils" }
ic_consensus_threshold_sig_system_test_utils = { path = "../tecdsa/utils" }
ic_bls12_381 = { workspace = true }
slog = { workspace = true }
tokio = { workspace = true }

Expand Down
Loading

0 comments on commit 6e32786

Please sign in to comment.