Change getHealth to compare optimistically confirmed slots (#33651)

The current getHealth mechanism checks a local accounts hash slot vs.
those of other nodes as specified by --known-validator. This is a
very coarse comparison given that the default for this value is 100
slots. More so, any nodes using a value larger than the default
(ie --incremental-snapshot-interval 500) will likely see getHealth
return status behind at some point.

Change the underlying mechanism of how health is computed. Instead of
using the accounts hash slots published in gossip, use the latest
optimistically confirmed slot from the cluster. Even when a node is
behind, it is able to observe cluster optimistically confirmed by slots
by viewing votes published in gossip.

Thus, the latest cluster optimistically confirmed slot can be compared
against the latest optimistically confirmed bank from replay to
determine health. This new comparison is much more granular, and not
needing to depend on individual known validators is also a plus.
This commit is contained in:
steviez 2023-10-16 11:21:33 -05:00 committed by GitHub
parent f4fb957a38
commit 8bd0e4cd95
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 213 additions and 238 deletions

View File

@ -943,7 +943,8 @@ impl Validator {
// (by both replay stage and banking stage) // (by both replay stage and banking stage)
let prioritization_fee_cache = Arc::new(PrioritizationFeeCache::default()); let prioritization_fee_cache = Arc::new(PrioritizationFeeCache::default());
let rpc_override_health_check = Arc::new(AtomicBool::new(false)); let rpc_override_health_check =
Arc::new(AtomicBool::new(config.rpc_config.disable_health_check));
let ( let (
json_rpc_service, json_rpc_service,
pubsub_service, pubsub_service,
@ -980,7 +981,6 @@ impl Validator {
ledger_path, ledger_path,
config.validator_exit.clone(), config.validator_exit.clone(),
exit.clone(), exit.clone(),
config.known_validators.clone(),
rpc_override_health_check.clone(), rpc_override_health_check.clone(),
startup_verification_complete, startup_verification_complete,
optimistically_confirmed_bank.clone(), optimistically_confirmed_bank.clone(),

View File

@ -154,13 +154,11 @@ Some methods support providing a `filters` object to enable pre-filtering the da
Although not a JSON RPC API, a `GET /health` at the RPC HTTP Endpoint provides a Although not a JSON RPC API, a `GET /health` at the RPC HTTP Endpoint provides a
health-check mechanism for use by load balancers or other network health-check mechanism for use by load balancers or other network
infrastructure. This request will always return a HTTP 200 OK response with a body of infrastructure. This request will always return a HTTP 200 OK response with a body of
"ok", "behind" or "unknown" based on the following conditions: "ok", "behind" or "unknown":
1. If one or more `--known-validator` arguments are provided to `solana-validator` - "ok" is returned - `ok`: The node is within `HEALTH_CHECK_SLOT_DISTANCE` slots from the latest cluster confirmed slot
when the node has within `HEALTH_CHECK_SLOT_DISTANCE` slots of the highest - `behind { distance }`: The node is behind `distance` slots from the latest cluster confirmed slot where `distance > HEALTH_CHECK_SLOT_DISTANCE`
known validator, otherwise "behind". "unknown" is returned when no slot - `unknown`: The node is unable to determine where it stands in relation to the cluster
information from known validators is not yet available.
2. "ok" is always returned if no known validators are provided.
## JSON RPC API Reference ## JSON RPC API Reference

View File

@ -12,13 +12,8 @@ import {
## getHealth ## getHealth
Returns the current health of the node. Returns the current health of the node. A healthy node is one that is within
`HEALTH_CHECK_SLOT_DISTANCE` slots of the latest cluster confirmed slot.
:::caution
If one or more `--known-validator` arguments are provided to `solana-validator` - "ok" is returned
when the node has within `HEALTH_CHECK_SLOT_DISTANCE` slots of the highest known validator,
otherwise an error is returned. "ok" is always returned if no known validators are provided.
:::
<DocSideBySide> <DocSideBySide>
<CodeParams> <CodeParams>

View File

@ -149,12 +149,15 @@ pub struct JsonRpcConfig {
pub obsolete_v1_7_api: bool, pub obsolete_v1_7_api: bool,
pub rpc_scan_and_fix_roots: bool, pub rpc_scan_and_fix_roots: bool,
pub max_request_body_size: Option<usize>, pub max_request_body_size: Option<usize>,
/// Disable the health check, used for tests and TestValidator
pub disable_health_check: bool,
} }
impl JsonRpcConfig { impl JsonRpcConfig {
pub fn default_for_test() -> Self { pub fn default_for_test() -> Self {
Self { Self {
full_api: true, full_api: true,
disable_health_check: true,
..Self::default() ..Self::default()
} }
} }
@ -374,6 +377,10 @@ impl JsonRpcRequestProcessor {
); );
let leader_schedule_cache = Arc::new(LeaderScheduleCache::new_from_bank(&bank)); let leader_schedule_cache = Arc::new(LeaderScheduleCache::new_from_bank(&bank));
let startup_verification_complete = Arc::clone(bank.get_startup_verification_complete());
let slot = bank.slot();
let optimistically_confirmed_bank =
Arc::new(RwLock::new(OptimisticallyConfirmedBank { bank }));
Self { Self {
config: JsonRpcConfig::default(), config: JsonRpcConfig::default(),
snapshot_config: None, snapshot_config: None,
@ -381,24 +388,22 @@ impl JsonRpcRequestProcessor {
block_commitment_cache: Arc::new(RwLock::new(BlockCommitmentCache::new( block_commitment_cache: Arc::new(RwLock::new(BlockCommitmentCache::new(
HashMap::new(), HashMap::new(),
0, 0,
CommitmentSlots::new_from_slot(bank.slot()), CommitmentSlots::new_from_slot(slot),
))), ))),
blockstore, blockstore: Arc::clone(&blockstore),
validator_exit: create_validator_exit(exit.clone()), validator_exit: create_validator_exit(exit.clone()),
health: Arc::new(RpcHealth::new( health: Arc::new(RpcHealth::new(
cluster_info.clone(), Arc::clone(&optimistically_confirmed_bank),
None, blockstore,
0, 0,
exit, exit,
Arc::clone(bank.get_startup_verification_complete()), startup_verification_complete,
)), )),
cluster_info, cluster_info,
genesis_hash, genesis_hash,
transaction_sender: Arc::new(Mutex::new(sender)), transaction_sender: Arc::new(Mutex::new(sender)),
bigtable_ledger_storage: None, bigtable_ledger_storage: None,
optimistically_confirmed_bank: Arc::new(RwLock::new(OptimisticallyConfirmedBank { optimistically_confirmed_bank,
bank,
})),
largest_accounts_cache: Arc::new(RwLock::new(LargestAccountsCache::new(30))), largest_accounts_cache: Arc::new(RwLock::new(LargestAccountsCache::new(30))),
max_slots: Arc::new(MaxSlots::default()), max_slots: Arc::new(MaxSlots::default()),
leader_schedule_cache, leader_schedule_cache,
@ -4787,6 +4792,8 @@ pub mod tests {
// note that this means that slot 0 will always be considered complete // note that this means that slot 0 will always be considered complete
let max_complete_transaction_status_slot = Arc::new(AtomicU64::new(0)); let max_complete_transaction_status_slot = Arc::new(AtomicU64::new(0));
let max_complete_rewards_slot = Arc::new(AtomicU64::new(0)); let max_complete_rewards_slot = Arc::new(AtomicU64::new(0));
let optimistically_confirmed_bank =
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
let meta = JsonRpcRequestProcessor::new( let meta = JsonRpcRequestProcessor::new(
config, config,
@ -4795,11 +4802,11 @@ pub mod tests {
block_commitment_cache.clone(), block_commitment_cache.clone(),
blockstore.clone(), blockstore.clone(),
validator_exit, validator_exit,
RpcHealth::stub(), RpcHealth::stub(optimistically_confirmed_bank.clone(), blockstore.clone()),
cluster_info, cluster_info,
Hash::default(), Hash::default(),
None, None,
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks), optimistically_confirmed_bank,
Arc::new(RwLock::new(LargestAccountsCache::new(30))), Arc::new(RwLock::new(LargestAccountsCache::new(30))),
max_slots.clone(), max_slots.clone(),
Arc::new(LeaderScheduleCache::new_from_bank(&bank)), Arc::new(LeaderScheduleCache::new_from_bank(&bank)),
@ -6398,7 +6405,11 @@ pub mod tests {
let blockstore = Arc::new(Blockstore::open(&ledger_path).unwrap()); let blockstore = Arc::new(Blockstore::open(&ledger_path).unwrap());
let block_commitment_cache = Arc::new(RwLock::new(BlockCommitmentCache::default())); let block_commitment_cache = Arc::new(RwLock::new(BlockCommitmentCache::default()));
let (bank_forks, mint_keypair, ..) = new_bank_forks(); let (bank_forks, mint_keypair, ..) = new_bank_forks();
let health = RpcHealth::stub(); let optimistically_confirmed_bank =
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
let health = RpcHealth::stub(optimistically_confirmed_bank.clone(), blockstore.clone());
// Mark the node as healthy to start
health.stub_set_health_status(Some(RpcHealthStatus::Ok));
// Freeze bank 0 to prevent a panic in `run_transaction_simulation()` // Freeze bank 0 to prevent a panic in `run_transaction_simulation()`
bank_forks.write().unwrap().get(0).unwrap().freeze(); bank_forks.write().unwrap().get(0).unwrap().freeze();
@ -6429,7 +6440,7 @@ pub mod tests {
cluster_info, cluster_info,
Hash::default(), Hash::default(),
None, None,
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks), optimistically_confirmed_bank,
Arc::new(RwLock::new(LargestAccountsCache::new(30))), Arc::new(RwLock::new(LargestAccountsCache::new(30))),
Arc::new(MaxSlots::default()), Arc::new(MaxSlots::default()),
Arc::new(LeaderScheduleCache::default()), Arc::new(LeaderScheduleCache::default()),
@ -6690,18 +6701,20 @@ pub mod tests {
.my_contact_info() .my_contact_info()
.tpu(connection_cache.protocol()) .tpu(connection_cache.protocol())
.unwrap(); .unwrap();
let optimistically_confirmed_bank =
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
let (request_processor, receiver) = JsonRpcRequestProcessor::new( let (request_processor, receiver) = JsonRpcRequestProcessor::new(
JsonRpcConfig::default(), JsonRpcConfig::default(),
None, None,
bank_forks.clone(), bank_forks.clone(),
block_commitment_cache, block_commitment_cache,
blockstore, blockstore.clone(),
validator_exit, validator_exit,
RpcHealth::stub(), RpcHealth::stub(optimistically_confirmed_bank.clone(), blockstore),
cluster_info, cluster_info,
Hash::default(), Hash::default(),
None, None,
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks), optimistically_confirmed_bank,
Arc::new(RwLock::new(LargestAccountsCache::new(30))), Arc::new(RwLock::new(LargestAccountsCache::new(30))),
Arc::new(MaxSlots::default()), Arc::new(MaxSlots::default()),
Arc::new(LeaderScheduleCache::default()), Arc::new(LeaderScheduleCache::default()),
@ -8327,9 +8340,9 @@ pub mod tests {
None, None,
bank_forks.clone(), bank_forks.clone(),
block_commitment_cache, block_commitment_cache,
blockstore, blockstore.clone(),
validator_exit, validator_exit,
RpcHealth::stub(), RpcHealth::stub(optimistically_confirmed_bank.clone(), blockstore.clone()),
cluster_info, cluster_info,
Hash::default(), Hash::default(),
None, None,

View File

@ -1,12 +1,10 @@
use { use {
solana_gossip::cluster_info::ClusterInfo, crate::optimistically_confirmed_bank_tracker::OptimisticallyConfirmedBank,
solana_sdk::{clock::Slot, pubkey::Pubkey}, solana_ledger::blockstore::Blockstore,
std::{ solana_sdk::clock::Slot,
collections::HashSet, std::sync::{
sync::{ atomic::{AtomicBool, Ordering},
atomic::{AtomicBool, Ordering}, Arc, RwLock,
Arc,
},
}, },
}; };
@ -18,8 +16,8 @@ pub enum RpcHealthStatus {
} }
pub struct RpcHealth { pub struct RpcHealth {
cluster_info: Arc<ClusterInfo>, optimistically_confirmed_bank: Arc<RwLock<OptimisticallyConfirmedBank>>,
known_validators: Option<HashSet<Pubkey>>, blockstore: Arc<Blockstore>,
health_check_slot_distance: u64, health_check_slot_distance: u64,
override_health_check: Arc<AtomicBool>, override_health_check: Arc<AtomicBool>,
startup_verification_complete: Arc<AtomicBool>, startup_verification_complete: Arc<AtomicBool>,
@ -29,15 +27,15 @@ pub struct RpcHealth {
impl RpcHealth { impl RpcHealth {
pub fn new( pub fn new(
cluster_info: Arc<ClusterInfo>, optimistically_confirmed_bank: Arc<RwLock<OptimisticallyConfirmedBank>>,
known_validators: Option<HashSet<Pubkey>>, blockstore: Arc<Blockstore>,
health_check_slot_distance: u64, health_check_slot_distance: u64,
override_health_check: Arc<AtomicBool>, override_health_check: Arc<AtomicBool>,
startup_verification_complete: Arc<AtomicBool>, startup_verification_complete: Arc<AtomicBool>,
) -> Self { ) -> Self {
Self { Self {
cluster_info, optimistically_confirmed_bank,
known_validators, blockstore,
health_check_slot_distance, health_check_slot_distance,
override_health_check, override_health_check,
startup_verification_complete, startup_verification_complete,
@ -54,84 +52,74 @@ impl RpcHealth {
} }
} }
if self.override_health_check.load(Ordering::Relaxed) {
return RpcHealthStatus::Ok;
}
if !self.startup_verification_complete.load(Ordering::Acquire) { if !self.startup_verification_complete.load(Ordering::Acquire) {
return RpcHealthStatus::Unknown; return RpcHealthStatus::Unknown;
} }
if self.override_health_check.load(Ordering::Relaxed) { // A node can observe votes by both replaying blocks and observing gossip.
RpcHealthStatus::Ok //
} else if let Some(known_validators) = &self.known_validators { // ClusterInfoVoteListener receives votes from both of these sources and then records
match ( // optimistically confirmed slots in the Blockstore via OptimisticConfirmationVerifier.
self.cluster_info // Thus, it is possible for a node to record an optimistically confirmed slot before the
.get_accounts_hash_for_node(&self.cluster_info.id(), |hashes| { // node has replayed and validated the slot for itself.
hashes //
.iter() // OptimisticallyConfirmedBank holds a bank for the latest optimistically confirmed slot
.max_by(|a, b| a.0.cmp(&b.0)) // that the node has replayed. It is true that the node will have replayed that slot by
.map(|slot_hash| slot_hash.0) // virtue of having a bank available. Observing that the cluster has optimistically
}) // confirmed a slot through gossip is not enough to reconstruct the bank.
.flatten(), //
known_validators // So, comparing the latest optimistic slot from the Blockstore vs. the slot from the
.iter() // OptimisticallyConfirmedBank bank allows a node to see where it stands in relation to the
.filter_map(|known_validator| { // tip of the cluster.
self.cluster_info let my_latest_optimistically_confirmed_slot = self
.get_accounts_hash_for_node(known_validator, |hashes| { .optimistically_confirmed_bank
hashes .read()
.iter() .unwrap()
.max_by(|a, b| a.0.cmp(&b.0)) .bank
.map(|slot_hash| slot_hash.0) .slot();
})
.flatten() let mut optimistic_slot_infos = match self.blockstore.get_latest_optimistic_slots(1) {
}) Ok(infos) => infos,
.max(), Err(err) => {
) { warn!("health check: blockstore error: {err}");
( return RpcHealthStatus::Unknown;
Some(latest_account_hash_slot),
Some(latest_known_validator_account_hash_slot),
) => {
// The validator is considered healthy if its latest account hash slot is within
// `health_check_slot_distance` of the latest known validator's account hash slot
if latest_account_hash_slot
> latest_known_validator_account_hash_slot
.saturating_sub(self.health_check_slot_distance)
{
RpcHealthStatus::Ok
} else {
let num_slots = latest_known_validator_account_hash_slot
.saturating_sub(latest_account_hash_slot);
warn!(
"health check: behind by {} slots: me={}, latest known_validator={}",
num_slots,
latest_account_hash_slot,
latest_known_validator_account_hash_slot
);
RpcHealthStatus::Behind { num_slots }
}
}
(latest_account_hash_slot, latest_known_validator_account_hash_slot) => {
if latest_account_hash_slot.is_none() {
warn!("health check: latest_account_hash_slot not available");
}
if latest_known_validator_account_hash_slot.is_none() {
warn!(
"health check: latest_known_validator_account_hash_slot not available"
);
}
RpcHealthStatus::Unknown
}
} }
} else { };
// No known validator point of reference available, so this validator is healthy let Some((cluster_latest_optimistically_confirmed_slot, _, _)) =
// because it's running optimistic_slot_infos.pop()
else {
warn!("health check: blockstore does not contain any optimistically confirmed slots");
return RpcHealthStatus::Unknown;
};
if my_latest_optimistically_confirmed_slot
>= cluster_latest_optimistically_confirmed_slot
.saturating_sub(self.health_check_slot_distance)
{
RpcHealthStatus::Ok RpcHealthStatus::Ok
} else {
let num_slots = cluster_latest_optimistically_confirmed_slot
.saturating_sub(my_latest_optimistically_confirmed_slot);
warn!(
"health check: behind by {num_slots} \
slots: me={my_latest_optimistically_confirmed_slot}, \
latest cluster={cluster_latest_optimistically_confirmed_slot}",
);
RpcHealthStatus::Behind { num_slots }
} }
} }
#[cfg(test)] #[cfg(test)]
pub(crate) fn stub() -> Arc<Self> { pub(crate) fn stub(
use crate::rpc::tests::new_test_cluster_info; optimistically_confirmed_bank: Arc<RwLock<OptimisticallyConfirmedBank>>,
blockstore: Arc<Blockstore>,
) -> Arc<Self> {
Arc::new(Self::new( Arc::new(Self::new(
Arc::new(new_test_cluster_info()), optimistically_confirmed_bank,
None, blockstore,
42, 42,
Arc::new(AtomicBool::new(false)), Arc::new(AtomicBool::new(false)),
Arc::new(AtomicBool::new(true)), Arc::new(AtomicBool::new(true)),
@ -143,3 +131,84 @@ impl RpcHealth {
*self.stub_health_status.write().unwrap() = stub_health_status; *self.stub_health_status.write().unwrap() = stub_health_status;
} }
} }
#[cfg(test)]
pub mod tests {
use {
super::*,
solana_ledger::{
genesis_utils::{create_genesis_config, GenesisConfigInfo},
get_tmp_ledger_path_auto_delete,
},
solana_runtime::{bank::Bank, bank_forks::BankForks},
solana_sdk::{clock::UnixTimestamp, hash::Hash, pubkey::Pubkey},
};
#[test]
fn test_get_health() {
let ledger_path = get_tmp_ledger_path_auto_delete!();
let blockstore = Arc::new(Blockstore::open(ledger_path.path()).unwrap());
let GenesisConfigInfo { genesis_config, .. } = create_genesis_config(100);
let bank = Bank::new_for_tests(&genesis_config);
let bank_forks = Arc::new(RwLock::new(BankForks::new(bank)));
let optimistically_confirmed_bank =
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
let bank0 = bank_forks.read().unwrap().root_bank();
assert!(bank0.slot() == 0);
let health_check_slot_distance = 10;
let override_health_check = Arc::new(AtomicBool::new(true));
let startup_verification_complete = Arc::clone(bank0.get_startup_verification_complete());
let health = RpcHealth::new(
optimistically_confirmed_bank.clone(),
blockstore.clone(),
health_check_slot_distance,
override_health_check.clone(),
startup_verification_complete,
);
// Override health check set to true - status is ok
assert_eq!(health.check(), RpcHealthStatus::Ok);
// Remove the override - status now unknown with incomplete startup verification
override_health_check.store(false, Ordering::Relaxed);
assert_eq!(health.check(), RpcHealthStatus::Unknown);
// Mark startup verification complete - status still unknown as no slots have been
// optimistically confirmed yet
bank0.set_startup_verification_complete();
assert_eq!(health.check(), RpcHealthStatus::Unknown);
// Mark slot 15 as being optimistically confirmed in the Blockstore, this could
// happen if the cluster confirmed the slot and this node became aware through gossip,
// but this node has not yet replayed slot 15. The local view of the latest optimistic
// slot is still slot 0 so status will be behind
blockstore
.insert_optimistic_slot(15, &Hash::default(), UnixTimestamp::default())
.unwrap();
assert_eq!(health.check(), RpcHealthStatus::Behind { num_slots: 15 });
// Simulate this node observing slot 4 as optimistically confirmed - status still behind
let bank4 = Arc::new(Bank::new_from_parent(bank0, &Pubkey::default(), 4));
optimistically_confirmed_bank.write().unwrap().bank = bank4.clone();
assert_eq!(health.check(), RpcHealthStatus::Behind { num_slots: 11 });
// Simulate this node observing slot 5 as optimistically confirmed - status now ok
// as distance is <= health_check_slot_distance
let bank5 = Arc::new(Bank::new_from_parent(bank4, &Pubkey::default(), 5));
optimistically_confirmed_bank.write().unwrap().bank = bank5.clone();
assert_eq!(health.check(), RpcHealthStatus::Ok);
// Node now up with tip of cluster
let bank15 = Arc::new(Bank::new_from_parent(bank5, &Pubkey::default(), 15));
optimistically_confirmed_bank.write().unwrap().bank = bank15.clone();
assert_eq!(health.check(), RpcHealthStatus::Ok);
// Node "beyond" tip of cluster - this technically isn't possible but could be
// observed locally due to a race between updates to Blockstore and
// OptimisticallyConfirmedBank. Either way, not a problem and status is ok.
let bank16 = Arc::new(Bank::new_from_parent(bank15, &Pubkey::default(), 16));
optimistically_confirmed_bank.write().unwrap().bank = bank16.clone();
assert_eq!(health.check(), RpcHealthStatus::Ok);
}
}

View File

@ -37,12 +37,11 @@ use {
}, },
solana_sdk::{ solana_sdk::{
exit::Exit, genesis_config::DEFAULT_GENESIS_DOWNLOAD_PATH, hash::Hash, exit::Exit, genesis_config::DEFAULT_GENESIS_DOWNLOAD_PATH, hash::Hash,
native_token::lamports_to_sol, pubkey::Pubkey, native_token::lamports_to_sol,
}, },
solana_send_transaction_service::send_transaction_service::{self, SendTransactionService}, solana_send_transaction_service::send_transaction_service::{self, SendTransactionService},
solana_storage_bigtable::CredentialType, solana_storage_bigtable::CredentialType,
std::{ std::{
collections::HashSet,
net::SocketAddr, net::SocketAddr,
path::{Path, PathBuf}, path::{Path, PathBuf},
sync::{ sync::{
@ -350,7 +349,6 @@ impl JsonRpcService {
ledger_path: &Path, ledger_path: &Path,
validator_exit: Arc<RwLock<Exit>>, validator_exit: Arc<RwLock<Exit>>,
exit: Arc<AtomicBool>, exit: Arc<AtomicBool>,
known_validators: Option<HashSet<Pubkey>>,
override_health_check: Arc<AtomicBool>, override_health_check: Arc<AtomicBool>,
startup_verification_complete: Arc<AtomicBool>, startup_verification_complete: Arc<AtomicBool>,
optimistically_confirmed_bank: Arc<RwLock<OptimisticallyConfirmedBank>>, optimistically_confirmed_bank: Arc<RwLock<OptimisticallyConfirmedBank>>,
@ -368,8 +366,8 @@ impl JsonRpcService {
let rpc_niceness_adj = config.rpc_niceness_adj; let rpc_niceness_adj = config.rpc_niceness_adj;
let health = Arc::new(RpcHealth::new( let health = Arc::new(RpcHealth::new(
cluster_info.clone(), Arc::clone(&optimistically_confirmed_bank),
known_validators, Arc::clone(&blockstore),
config.health_check_slot_distance, config.health_check_slot_distance,
override_health_check, override_health_check,
startup_verification_complete, startup_verification_complete,
@ -586,10 +584,6 @@ mod tests {
use { use {
super::*, super::*,
crate::rpc::{create_validator_exit, tests::new_test_cluster_info}, crate::rpc::{create_validator_exit, tests::new_test_cluster_info},
solana_gossip::{
crds::GossipRoute,
crds_value::{AccountsHashes, CrdsData, CrdsValue},
},
solana_ledger::{ solana_ledger::{
genesis_utils::{create_genesis_config, GenesisConfigInfo}, genesis_utils::{create_genesis_config, GenesisConfigInfo},
get_tmp_ledger_path, get_tmp_ledger_path,
@ -643,7 +637,6 @@ mod tests {
&PathBuf::from("farf"), &PathBuf::from("farf"),
validator_exit, validator_exit,
exit, exit,
None,
Arc::new(AtomicBool::new(false)), Arc::new(AtomicBool::new(false)),
Arc::new(AtomicBool::new(true)), Arc::new(AtomicBool::new(true)),
optimistically_confirmed_bank, optimistically_confirmed_bank,
@ -726,18 +719,25 @@ mod tests {
#[test] #[test]
fn test_is_file_get_path() { fn test_is_file_get_path() {
let ledger_path = get_tmp_ledger_path!();
let blockstore = Arc::new(Blockstore::open(&ledger_path).unwrap());
let bank_forks = create_bank_forks();
let optimistically_confirmed_bank =
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
let health = RpcHealth::stub(optimistically_confirmed_bank, blockstore);
let bank_forks = create_bank_forks(); let bank_forks = create_bank_forks();
let rrm = RpcRequestMiddleware::new( let rrm = RpcRequestMiddleware::new(
PathBuf::from("/"), ledger_path.clone(),
None, None,
bank_forks.clone(), bank_forks.clone(),
RpcHealth::stub(), health.clone(),
); );
let rrm_with_snapshot_config = RpcRequestMiddleware::new( let rrm_with_snapshot_config = RpcRequestMiddleware::new(
PathBuf::from("/"), ledger_path.clone(),
Some(SnapshotConfig::default()), Some(SnapshotConfig::default()),
bank_forks, bank_forks,
RpcHealth::stub(), health,
); );
assert!(rrm.is_file_get_path(DEFAULT_GENESIS_DOWNLOAD_PATH)); assert!(rrm.is_file_get_path(DEFAULT_GENESIS_DOWNLOAD_PATH));
@ -830,14 +830,17 @@ mod tests {
let runtime = Runtime::new().unwrap(); let runtime = Runtime::new().unwrap();
let ledger_path = get_tmp_ledger_path!(); let ledger_path = get_tmp_ledger_path!();
std::fs::create_dir(&ledger_path).unwrap(); let blockstore = Arc::new(Blockstore::open(&ledger_path).unwrap());
let genesis_path = ledger_path.join(DEFAULT_GENESIS_ARCHIVE); let genesis_path = ledger_path.join(DEFAULT_GENESIS_ARCHIVE);
let bank_forks = create_bank_forks();
let optimistically_confirmed_bank =
OptimisticallyConfirmedBank::locked_from_bank_forks_root(&bank_forks);
let rrm = RpcRequestMiddleware::new( let rrm = RpcRequestMiddleware::new(
ledger_path.clone(), ledger_path.clone(),
None, None,
create_bank_forks(), bank_forks,
RpcHealth::stub(), RpcHealth::stub(optimistically_confirmed_bank, blockstore),
); );
// File does not exist => request should fail. // File does not exist => request should fail.
@ -885,106 +888,4 @@ mod tests {
} }
} }
} }
#[test]
fn test_health_check_with_no_known_validators() {
let rm = RpcRequestMiddleware::new(
PathBuf::from("/"),
None,
create_bank_forks(),
RpcHealth::stub(),
);
assert_eq!(rm.health_check(), "ok");
}
#[test]
fn test_health_check_with_known_validators() {
let cluster_info = Arc::new(new_test_cluster_info());
let health_check_slot_distance = 123;
let override_health_check = Arc::new(AtomicBool::new(false));
let startup_verification_complete = Arc::new(AtomicBool::new(true));
let known_validators = vec![
solana_sdk::pubkey::new_rand(),
solana_sdk::pubkey::new_rand(),
solana_sdk::pubkey::new_rand(),
];
let health = Arc::new(RpcHealth::new(
cluster_info.clone(),
Some(known_validators.clone().into_iter().collect()),
health_check_slot_distance,
override_health_check.clone(),
startup_verification_complete,
));
let rm = RpcRequestMiddleware::new(PathBuf::from("/"), None, create_bank_forks(), health);
// No account hashes for this node or any known validators
assert_eq!(rm.health_check(), "unknown");
// No account hashes for any known validators
cluster_info.push_accounts_hashes(vec![(1000, Hash::default()), (900, Hash::default())]);
cluster_info.flush_push_queue();
assert_eq!(rm.health_check(), "unknown");
// Override health check
override_health_check.store(true, Ordering::Relaxed);
assert_eq!(rm.health_check(), "ok");
override_health_check.store(false, Ordering::Relaxed);
// This node is ahead of the known validators
cluster_info
.gossip
.crds
.write()
.unwrap()
.insert(
CrdsValue::new_unsigned(CrdsData::AccountsHashes(AccountsHashes::new(
known_validators[0],
vec![
(1, Hash::default()),
(1001, Hash::default()),
(2, Hash::default()),
],
))),
1,
GossipRoute::LocalMessage,
)
.unwrap();
assert_eq!(rm.health_check(), "ok");
// Node is slightly behind the known validators
cluster_info
.gossip
.crds
.write()
.unwrap()
.insert(
CrdsValue::new_unsigned(CrdsData::AccountsHashes(AccountsHashes::new(
known_validators[1],
vec![(1000 + health_check_slot_distance - 1, Hash::default())],
))),
1,
GossipRoute::LocalMessage,
)
.unwrap();
assert_eq!(rm.health_check(), "ok");
// Node is far behind the known validators
cluster_info
.gossip
.crds
.write()
.unwrap()
.insert(
CrdsValue::new_unsigned(CrdsData::AccountsHashes(AccountsHashes::new(
known_validators[2],
vec![(1000 + health_check_slot_distance, Hash::default())],
))),
1,
GossipRoute::LocalMessage,
)
.unwrap();
assert_eq!(rm.health_check(), "behind");
}
} }

View File

@ -258,11 +258,9 @@ pub fn app<'a>(version: &'a str, default_args: &'a DefaultArgs) -> App<'a, 'a> {
.value_name("SLOT_DISTANCE") .value_name("SLOT_DISTANCE")
.takes_value(true) .takes_value(true)
.default_value(&default_args.health_check_slot_distance) .default_value(&default_args.health_check_slot_distance)
.help("If --known-validators are specified, report this validator healthy \ .help("Report this validator healthy if its latest optimistically confirmed slot \
if its latest account hash is no further behind than this number of \ that has been replayed is no further behind than this number of slots from \
slots from the latest known validator account hash. \ the cluster latest optimistically confirmed slot")
If no --known-validators are specified, the validator will always \
report itself to be healthy")
) )
.arg( .arg(
Arg::with_name("rpc_faucet_addr") Arg::with_name("rpc_faucet_addr")

View File

@ -1299,6 +1299,7 @@ pub fn main() {
"health_check_slot_distance", "health_check_slot_distance",
u64 u64
), ),
disable_health_check: false,
rpc_threads: value_t_or_exit!(matches, "rpc_threads", usize), rpc_threads: value_t_or_exit!(matches, "rpc_threads", usize),
rpc_niceness_adj: value_t_or_exit!(matches, "rpc_niceness_adj", i8), rpc_niceness_adj: value_t_or_exit!(matches, "rpc_niceness_adj", i8),
account_indexes: account_indexes.clone(), account_indexes: account_indexes.clone(),