diff --git a/crates/example-types/src/node_types.rs b/crates/example-types/src/node_types.rs
index b45e2e7377..c29c4f898f 100644
--- a/crates/example-types/src/node_types.rs
+++ b/crates/example-types/src/node_types.rs
@@ -4,9 +4,14 @@
// You should have received a copy of the MIT License
// along with the HotShot repository. If not, see .
+use std::marker::PhantomData;
+
+pub use hotshot::traits::election::helpers::{
+ RandomOverlapQuorumFilterConfig, StableQuorumFilterConfig,
+};
use hotshot::traits::{
election::{
- randomized_committee::RandomizedCommittee,
+ helpers::QuorumFilterConfig, randomized_committee::RandomizedCommittee,
randomized_committee_members::RandomizedCommitteeMembers,
static_committee::StaticCommittee,
static_committee_leader_two_views::StaticCommitteeLeaderForTwoViews,
@@ -104,10 +109,11 @@ impl NodeType for TestTypesRandomizedLeader {
)]
/// filler struct to implement node type and allow us
/// to select our traits
-pub struct TestTypesRandomizedCommitteeMembers;
-impl NodeType
- for TestTypesRandomizedCommitteeMembers
-{
+pub struct TestTypesRandomizedCommitteeMembers {
+ _pd: PhantomData,
+}
+
+impl NodeType for TestTypesRandomizedCommitteeMembers {
type AuctionResult = TestAuctionResult;
type View = ViewNumber;
type Epoch = EpochNumber;
@@ -117,11 +123,8 @@ impl NodeType
type Transaction = TestTransaction;
type ValidatedState = TestValidatedState;
type InstanceState = TestInstanceState;
- type Membership = RandomizedCommitteeMembers<
- TestTypesRandomizedCommitteeMembers,
- SEED,
- OVERLAP,
- >;
+ type Membership =
+ RandomizedCommitteeMembers, CONFIG>;
type BuilderSignatureKey = BuilderKey;
}
diff --git a/crates/examples/push-cdn/all.rs b/crates/examples/push-cdn/all.rs
index 2e04d2d72b..12599c36a4 100644
--- a/crates/examples/push-cdn/all.rs
+++ b/crates/examples/push-cdn/all.rs
@@ -14,7 +14,6 @@ use cdn_broker::{
reexports::{crypto::signature::KeyPair, def::hook::NoMessageHook},
Broker,
};
-
use cdn_marshal::Marshal;
use hotshot::{
helpers::initialize_logging,
diff --git a/crates/hotshot/src/tasks/mod.rs b/crates/hotshot/src/tasks/mod.rs
index 6ddd00b252..bb9f7ed630 100644
--- a/crates/hotshot/src/tasks/mod.rs
+++ b/crates/hotshot/src/tasks/mod.rs
@@ -10,11 +10,6 @@
pub mod task_state;
use std::{collections::BTreeMap, fmt::Debug, num::NonZeroUsize, sync::Arc, time::Duration};
-use crate::{
- tasks::task_state::CreateTaskState, types::SystemContextHandle, ConsensusApi,
- ConsensusMetricsValue, ConsensusTaskRegistry, HotShotConfig, HotShotInitializer,
- MarketplaceConfig, Memberships, NetworkTaskRegistry, SignatureKey, SystemContext, Versions,
-};
use async_broadcast::{broadcast, RecvError};
use async_lock::RwLock;
use async_trait::async_trait;
@@ -48,6 +43,12 @@ use hotshot_types::{
use tokio::{spawn, time::sleep};
use vbs::version::StaticVersionType;
+use crate::{
+ tasks::task_state::CreateTaskState, types::SystemContextHandle, ConsensusApi,
+ ConsensusMetricsValue, ConsensusTaskRegistry, HotShotConfig, HotShotInitializer,
+ MarketplaceConfig, Memberships, NetworkTaskRegistry, SignatureKey, SystemContext, Versions,
+};
+
/// event for global event stream
#[derive(Clone, Debug)]
pub enum GlobalEvent {
diff --git a/crates/hotshot/src/traits/election/helpers.rs b/crates/hotshot/src/traits/election/helpers.rs
index 334458f57d..5d2a62ecb5 100644
--- a/crates/hotshot/src/traits/election/helpers.rs
+++ b/crates/hotshot/src/traits/election/helpers.rs
@@ -4,7 +4,7 @@
// You should have received a copy of the MIT License
// along with the HotShot repository. If not, see .
-use std::collections::BTreeSet;
+use std::{collections::BTreeSet, hash::Hash};
use rand::{rngs::StdRng, Rng, SeedableRng};
@@ -52,7 +52,11 @@ impl Iterator for NonRepeatValueIterator {
}
/// Create a single u64 seed by merging two u64s. Done this way to allow easy seeding of the number generator
-/// from both a stable SOUND as well as a moving value ROUND (typically, epoch).
+/// from both a stable SOUND as well as a moving value ROUND (typically, epoch). Shift left by 8 to avoid
+/// scenarios where someone manually stepping seeds would pass over the same space of random numbers across
+/// sequential rounds. Doesn't have to be 8, but has to be large enough that it is unlikely that a given
+/// test run will collide; using 8 means that 256 rounds (epochs) would have to happen inside of a test before
+/// the test starts repeating values from SEED+1.
fn make_seed(seed: u64, round: u64) -> u64 {
seed.wrapping_add(round.wrapping_shl(8))
}
@@ -102,7 +106,12 @@ fn calc_num_slots(count: u64, odd: bool) -> u64 {
}
impl StableQuorumIterator {
+ #[must_use]
/// Create a new StableQuorumIterator
+ ///
+ /// # Panics
+ ///
+ /// panics if overlap is greater than half of count
pub fn new(seed: u64, round: u64, count: u64, overlap: u64) -> Self {
assert!(
count / 2 > overlap,
@@ -127,14 +136,20 @@ impl Iterator for StableQuorumIterator {
fn next(&mut self) -> Option {
if self.index >= (self.count / 2) {
+ // Always return exactly half of the possible values. If we have OVERLAP>0 then
+ // we need to return (COUNT/2)-OVERLAP of the current set, even if there are additional
+ // even (or odd) numbers that we can return.
None
} else if self.index < self.overlap {
- // Generate enough values for the previous round
+ // Generate enough values for the previous round. If the current round is odd, then
+ // we want to pick even values that were selected from the previous round to create OVERLAP
+ // even values.
let v = self.prev_rng.next().unwrap();
self.index += 1;
Some(v * 2 + self.round % 2)
} else {
- // Generate new values
+ // Generate new values. If our current round is odd, we'll be creating (COUNT/2)-OVERLAP
+ // odd values here.
let v = self.this_rng.next().unwrap();
self.index += 1;
Some(v * 2 + (1 - self.round % 2))
@@ -142,7 +157,12 @@ impl Iterator for StableQuorumIterator {
}
}
+#[must_use]
/// Helper function to convert the arguments to a StableQuorumIterator into an ordered set of values.
+///
+/// # Panics
+///
+/// panics if the arguments are invalid for StableQuorumIterator::new
pub fn stable_quorum_filter(seed: u64, round: u64, count: usize, overlap: u64) -> BTreeSet {
StableQuorumIterator::new(seed, round, count as u64, overlap)
// We should never have more than u32_max members in a test
@@ -175,7 +195,12 @@ pub struct RandomOverlapQuorumIterator {
}
impl RandomOverlapQuorumIterator {
+ #[must_use]
/// Create a new RandomOverlapQuorumIterator
+ ///
+ /// # Panics
+ ///
+ /// panics if overlap and members can produce invalid results or if ranges are invalid
pub fn new(
seed: u64,
round: u64,
@@ -231,16 +256,109 @@ impl Iterator for RandomOverlapQuorumIterator {
// Generate enough values for the previous round
let v = self.prev_rng.next().unwrap();
self.index += 1;
- Some(v * 2 + self.round % 2)
+ Some(v * 2 + (1 - self.round % 2))
} else {
// Generate new values
let v = self.this_rng.next().unwrap();
self.index += 1;
- Some(v * 2 + (1 - self.round % 2))
+ Some(v * 2 + self.round % 2)
}
}
}
+#[must_use]
+/// Helper function to convert the arguments to a StableQuorumIterator into an ordered set of values.
+///
+/// # Panics
+///
+/// panics if the arguments are invalid for RandomOverlapQuorumIterator::new
+pub fn random_overlap_quorum_filter(
+ seed: u64,
+ round: u64,
+ count: usize,
+ members_min: u64,
+ members_max: u64,
+ overlap_min: u64,
+ overlap_max: u64,
+) -> BTreeSet {
+ RandomOverlapQuorumIterator::new(
+ seed,
+ round,
+ count as u64,
+ members_min,
+ members_max,
+ overlap_min,
+ overlap_max,
+ )
+ // We should never have more than u32_max members in a test
+ .map(|x| usize::try_from(x).unwrap())
+ .collect()
+}
+
+/// Trait wrapping a config for quorum filters. This allows selection between either the StableQuorumIterator or the
+/// RandomOverlapQuorumIterator functionality from above
+pub trait QuorumFilterConfig:
+ Copy
+ + Clone
+ + std::fmt::Debug
+ + Default
+ + Send
+ + Sync
+ + Ord
+ + PartialOrd
+ + Eq
+ + PartialEq
+ + Hash
+ + 'static
+{
+ /// Called to run the filter and return a set of indices
+ fn execute(epoch: u64, count: usize) -> BTreeSet;
+}
+
+#[derive(Debug, Copy, Clone, Default, Eq, PartialEq, Hash, Ord, PartialOrd)]
+/// Provides parameters to use the StableQuorumIterator
+pub struct StableQuorumFilterConfig {}
+
+impl QuorumFilterConfig
+ for StableQuorumFilterConfig
+{
+ fn execute(epoch: u64, count: usize) -> BTreeSet {
+ stable_quorum_filter(SEED, epoch, count, OVERLAP)
+ }
+}
+
+#[derive(Debug, Copy, Clone, Default, Eq, PartialEq, Hash, Ord, PartialOrd)]
+/// Provides parameters to use the RandomOverlapQuorumIterator
+pub struct RandomOverlapQuorumFilterConfig<
+ const SEED: u64,
+ const MEMBERS_MIN: u64,
+ const MEMBERS_MAX: u64,
+ const OVERLAP_MIN: u64,
+ const OVERLAP_MAX: u64,
+> {}
+
+impl<
+ const SEED: u64,
+ const MEMBERS_MIN: u64,
+ const MEMBERS_MAX: u64,
+ const OVERLAP_MIN: u64,
+ const OVERLAP_MAX: u64,
+ > QuorumFilterConfig
+ for RandomOverlapQuorumFilterConfig
+{
+ fn execute(epoch: u64, count: usize) -> BTreeSet {
+ random_overlap_quorum_filter(
+ SEED,
+ epoch,
+ count,
+ MEMBERS_MIN,
+ MEMBERS_MAX,
+ OVERLAP_MIN,
+ OVERLAP_MAX,
+ )
+ }
+}
+
#[cfg(test)]
mod tests {
use super::*;
@@ -279,4 +397,37 @@ mod tests {
assert!(matched, "prev_set={prev_set:?}, this_set={this_set:?}");
}
}
+
+ #[test]
+ fn test_odd_even() {
+ for _ in 0..100 {
+ let seed = rand::random::();
+
+ let odd_set: Vec = StableQuorumIterator::new(seed, 1, 10, 2).collect();
+ let even_set: Vec = StableQuorumIterator::new(seed, 2, 10, 2).collect();
+
+ assert!(
+ odd_set[2] % 2 == 1,
+ "odd set non-overlap value should be odd (stable)"
+ );
+ assert!(
+ even_set[2] % 2 == 0,
+ "even set non-overlap value should be even (stable)"
+ );
+
+ let odd_set: Vec =
+ RandomOverlapQuorumIterator::new(seed, 1, 20, 5, 10, 2, 3).collect();
+ let even_set: Vec =
+ RandomOverlapQuorumIterator::new(seed, 2, 20, 5, 10, 2, 3).collect();
+
+ assert!(
+ odd_set[3] % 2 == 1,
+ "odd set non-overlap value should be odd (random overlap)"
+ );
+ assert!(
+ even_set[3] % 2 == 0,
+ "even set non-overlap value should be even (random overlap)"
+ );
+ }
+ }
}
diff --git a/crates/hotshot/src/traits/election/mod.rs b/crates/hotshot/src/traits/election/mod.rs
index 8020aa9d08..914b9bbb33 100644
--- a/crates/hotshot/src/traits/election/mod.rs
+++ b/crates/hotshot/src/traits/election/mod.rs
@@ -19,4 +19,4 @@ pub mod static_committee;
pub mod static_committee_leader_two_views;
/// general helpers
-mod helpers;
+pub mod helpers;
diff --git a/crates/hotshot/src/traits/election/randomized_committee_members.rs b/crates/hotshot/src/traits/election/randomized_committee_members.rs
index ab8f8a534e..8810adf956 100644
--- a/crates/hotshot/src/traits/election/randomized_committee_members.rs
+++ b/crates/hotshot/src/traits/election/randomized_committee_members.rs
@@ -7,6 +7,7 @@
use std::{
cmp::max,
collections::{BTreeMap, BTreeSet},
+ marker::PhantomData,
num::NonZeroU64,
};
@@ -23,13 +24,11 @@ use hotshot_types::{
use rand::{rngs::StdRng, Rng};
use utils::anytrace::Result;
-use super::helpers::stable_quorum_filter;
+use crate::traits::election::helpers::QuorumFilterConfig;
#[derive(Clone, Debug, Eq, PartialEq, Hash)]
-
/// The static committee election
-
-pub struct RandomizedCommitteeMembers {
+pub struct RandomizedCommitteeMembers {
/// The nodes eligible for leadership.
/// NOTE: This is currently a hack because the DA leader needs to be the quorum
/// leader but without voting rights.
@@ -44,19 +43,20 @@ pub struct RandomizedCommitteeMembers,
}
-impl
- RandomizedCommitteeMembers
-{
+impl RandomizedCommitteeMembers {
/// Creates a set of indices into the stake_table which reference the nodes selected for this epoch's committee
fn make_quorum_filter(&self, epoch: ::Epoch) -> BTreeSet {
- stable_quorum_filter(SEED, epoch.u64(), self.stake_table.len(), OVERLAP)
+ CONFIG::execute(epoch.u64(), self.stake_table.len())
}
}
-impl Membership
- for RandomizedCommitteeMembers
+impl Membership
+ for RandomizedCommitteeMembers
{
type Error = utils::anytrace::Error;
@@ -102,6 +102,7 @@ impl Membership
stake_table: members,
indexed_stake_table,
committee_topic,
+ _pd: PhantomData,
}
}
diff --git a/crates/macros/src/lib.rs b/crates/macros/src/lib.rs
index 2666b8fbca..80f49e1d6a 100644
--- a/crates/macros/src/lib.rs
+++ b/crates/macros/src/lib.rs
@@ -13,7 +13,7 @@ use syn::{
parse::{Parse, ParseStream, Result},
parse_macro_input,
punctuated::Punctuated,
- Expr, ExprArray, ExprPath, ExprTuple, Ident, LitBool, Token, TypePath,
+ Expr, ExprArray, ExprPath, ExprTuple, Ident, LitBool, PathArguments, Token, TypePath,
};
/// Bracketed types, e.g. [A, B, C]
@@ -108,6 +108,37 @@ impl ToLowerSnakeStr for ExprPath {
}
}
+impl ToLowerSnakeStr for syn::GenericArgument {
+ /// allow panic because this is a compiler error
+ #[allow(clippy::panic)]
+ fn to_lower_snake_str(&self) -> String {
+ match self {
+ syn::GenericArgument::Lifetime(l) => l.ident.to_string().to_lowercase(),
+ syn::GenericArgument::Type(t) => match t {
+ syn::Type::Path(p) => p.to_lower_snake_str(),
+ _ => {
+ panic!("Unexpected type for GenericArgument::Type: {t:?}");
+ }
+ },
+ syn::GenericArgument::Const(c) => match c {
+ syn::Expr::Lit(l) => match &l.lit {
+ syn::Lit::Str(v) => format!("{}_", v.value().to_lowercase()),
+ syn::Lit::Int(v) => format!("{}_", v.base10_digits()),
+ _ => {
+ panic!("Unexpected type for GenericArgument::Const::Lit: {l:?}");
+ }
+ },
+ _ => {
+ panic!("Unexpected type for GenericArgument::Const: {c:?}");
+ }
+ },
+ _ => {
+ panic!("Unexpected type for GenericArgument: {self:?}");
+ }
+ }
+ }
+}
+
impl ToLowerSnakeStr for TypePath {
fn to_lower_snake_str(&self) -> String {
self.path
@@ -115,6 +146,13 @@ impl ToLowerSnakeStr for TypePath {
.iter()
.fold(String::new(), |mut acc, s| {
acc.push_str(&s.ident.to_string().to_lowercase());
+ if let PathArguments::AngleBracketed(a) = &s.arguments {
+ acc.push('_');
+ for arg in &a.args {
+ acc.push_str(&arg.to_lower_snake_str());
+ }
+ }
+
acc.push('_');
acc
})
diff --git a/crates/task-impls/src/helpers.rs b/crates/task-impls/src/helpers.rs
index bcabd40e9e..a1c4c28a63 100644
--- a/crates/task-impls/src/helpers.rs
+++ b/crates/task-impls/src/helpers.rs
@@ -13,7 +13,6 @@ use async_broadcast::{Receiver, SendError, Sender};
use async_lock::RwLock;
use committable::{Commitment, Committable};
use hotshot_task::dependency::{Dependency, EventDependency};
-use hotshot_types::utils::epoch_from_block_number;
use hotshot_types::{
consensus::OuterConsensus,
data::{Leaf, QuorumProposal, ViewChangeEvidence},
@@ -28,7 +27,7 @@ use hotshot_types::{
signature_key::SignatureKey,
BlockPayload, ValidatedState,
},
- utils::{Terminator, View, ViewInner},
+ utils::{epoch_from_block_number, Terminator, View, ViewInner},
vote::{Certificate, HasViewNumber},
};
use tokio::time::timeout;
diff --git a/crates/task-impls/src/quorum_proposal_recv/handlers.rs b/crates/task-impls/src/quorum_proposal_recv/handlers.rs
index d7ce8aefc0..6d827c6c74 100644
--- a/crates/task-impls/src/quorum_proposal_recv/handlers.rs
+++ b/crates/task-impls/src/quorum_proposal_recv/handlers.rs
@@ -11,13 +11,13 @@ use std::sync::Arc;
use async_broadcast::{broadcast, Receiver, Sender};
use async_lock::RwLockUpgradableReadGuard;
use committable::Committable;
-use hotshot_types::traits::block_contents::BlockHeader;
use hotshot_types::{
consensus::OuterConsensus,
data::{Leaf, QuorumProposal},
message::Proposal,
simple_certificate::QuorumCertificate,
traits::{
+ block_contents::BlockHeader,
election::Membership,
node_implementation::{ConsensusTime, NodeImplementation, NodeType},
signature_key::SignatureKey,
diff --git a/crates/task-impls/src/quorum_proposal_recv/mod.rs b/crates/task-impls/src/quorum_proposal_recv/mod.rs
index 332ecaf241..78665d519f 100644
--- a/crates/task-impls/src/quorum_proposal_recv/mod.rs
+++ b/crates/task-impls/src/quorum_proposal_recv/mod.rs
@@ -8,11 +8,6 @@
use std::{collections::BTreeMap, sync::Arc};
-use self::handlers::handle_quorum_proposal_recv;
-use crate::{
- events::{HotShotEvent, ProposalMissing},
- helpers::{broadcast_event, fetch_proposal},
-};
use async_broadcast::{broadcast, Receiver, Sender};
use async_lock::RwLock;
use async_trait::async_trait;
@@ -35,6 +30,12 @@ use tokio::task::JoinHandle;
use tracing::{debug, error, info, instrument, warn};
use utils::anytrace::{bail, Result};
use vbs::version::Version;
+
+use self::handlers::handle_quorum_proposal_recv;
+use crate::{
+ events::{HotShotEvent, ProposalMissing},
+ helpers::{broadcast_event, fetch_proposal},
+};
/// Event handlers for this task.
mod handlers;
diff --git a/crates/task-impls/src/quorum_vote/mod.rs b/crates/task-impls/src/quorum_vote/mod.rs
index 6ac5fe37dd..0dfc13683c 100644
--- a/crates/task-impls/src/quorum_vote/mod.rs
+++ b/crates/task-impls/src/quorum_vote/mod.rs
@@ -6,11 +6,6 @@
use std::{collections::BTreeMap, sync::Arc};
-use crate::{
- events::HotShotEvent,
- helpers::broadcast_event,
- quorum_vote::handlers::{handle_quorum_proposal_validated, submit_vote, update_shared_state},
-};
use async_broadcast::{InactiveReceiver, Receiver, Sender};
use async_lock::RwLock;
use async_trait::async_trait;
@@ -41,6 +36,12 @@ use tracing::instrument;
use utils::anytrace::*;
use vbs::version::StaticVersionType;
+use crate::{
+ events::HotShotEvent,
+ helpers::broadcast_event,
+ quorum_vote::handlers::{handle_quorum_proposal_validated, submit_vote, update_shared_state},
+};
+
/// Event handlers for `QuorumProposalValidated`.
mod handlers;
diff --git a/crates/testing/tests/tests_1/block_builder.rs b/crates/testing/tests/tests_1/block_builder.rs
index 572741607a..fc29b1c01d 100644
--- a/crates/testing/tests/tests_1/block_builder.rs
+++ b/crates/testing/tests/tests_1/block_builder.rs
@@ -12,7 +12,7 @@ use std::{
use hotshot_builder_api::v0_1::block_info::AvailableBlockData;
use hotshot_example_types::{
block_types::{TestBlockPayload, TestMetadata, TestTransaction},
- node_types::{TestTypes, TestVersions},
+ node_types::TestTypes,
};
use hotshot_task_impls::builder::{BuilderClient, BuilderClientError};
use hotshot_testing::block_builder::{
diff --git a/crates/testing/tests/tests_1/quorum_proposal_recv_task.rs b/crates/testing/tests/tests_1/quorum_proposal_recv_task.rs
index dc42dfc723..430f589e75 100644
--- a/crates/testing/tests/tests_1/quorum_proposal_recv_task.rs
+++ b/crates/testing/tests/tests_1/quorum_proposal_recv_task.rs
@@ -27,9 +27,8 @@ use hotshot_testing::{
serial,
view_generator::TestViewGenerator,
};
-use hotshot_types::data::EpochNumber;
use hotshot_types::{
- data::{Leaf, ViewNumber},
+ data::{EpochNumber, Leaf, ViewNumber},
request_response::ProposalRequestPayload,
traits::{
consensus_api::ConsensusApi,
diff --git a/crates/testing/tests/tests_1/test_success.rs b/crates/testing/tests/tests_1/test_success.rs
index a40d865ed6..a19f904694 100644
--- a/crates/testing/tests/tests_1/test_success.rs
+++ b/crates/testing/tests/tests_1/test_success.rs
@@ -8,8 +8,9 @@ use std::time::Duration;
use hotshot_example_types::{
node_types::{
- EpochsTestVersions, Libp2pImpl, MemoryImpl, PushCdnImpl, TestConsecutiveLeaderTypes,
- TestTypes, TestTypesRandomizedCommitteeMembers, TestTypesRandomizedLeader, TestVersions,
+ EpochsTestVersions, Libp2pImpl, MemoryImpl, PushCdnImpl, StableQuorumFilterConfig,
+ TestConsecutiveLeaderTypes, TestTypes, TestTypesRandomizedCommitteeMembers,
+ TestTypesRandomizedLeader, TestVersions,
},
testable_delay::{DelayConfig, DelayOptions, DelaySettings, SupportedTraitTypesForAsyncDelay},
};
@@ -44,7 +45,7 @@ cross_tests!(
cross_tests!(
TestName: test_epoch_success,
Impls: [MemoryImpl, Libp2pImpl, PushCdnImpl],
- Types: [TestTypes, TestTypesRandomizedLeader, TestTypesRandomizedCommitteeMembers<123, 2>],
+ Types: [TestTypes, TestTypesRandomizedLeader, TestTypesRandomizedCommitteeMembers>],
Versions: [EpochsTestVersions],
Ignore: false,
Metadata: {
diff --git a/crates/testing/tests/tests_1/view_sync_task.rs b/crates/testing/tests/tests_1/view_sync_task.rs
index 85827dbbdb..ec313ec2a7 100644
--- a/crates/testing/tests/tests_1/view_sync_task.rs
+++ b/crates/testing/tests/tests_1/view_sync_task.rs
@@ -10,9 +10,9 @@ use hotshot_task_impls::{
events::HotShotEvent, harness::run_harness, view_sync::ViewSyncTaskState,
};
use hotshot_testing::helpers::build_system_handle;
-use hotshot_types::data::EpochNumber;
use hotshot_types::{
- data::ViewNumber, simple_vote::ViewSyncPreCommitData,
+ data::{EpochNumber, ViewNumber},
+ simple_vote::ViewSyncPreCommitData,
traits::node_implementation::ConsensusTime,
};