diff --git a/Cargo.lock b/Cargo.lock index 1cd4d4235b9..ad700caaa61 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3009,8 +3009,10 @@ dependencies = [ "chrono", "derive_more", "near-crypto", + "near-o11y", "near-primitives", "num-rational", + "once_cell", "serde", "serde_json", "sha2 0.10.2", @@ -3153,9 +3155,17 @@ dependencies = [ name = "near-dyn-configs" version = "0.0.0" dependencies = [ + "anyhow", + "near-chain-configs", "near-o11y", + "near-primitives", "once_cell", "prometheus", + "serde", + "serde_json", + "thiserror", + "tokio", + "tracing", ] [[package]] @@ -3868,6 +3878,7 @@ dependencies = [ "futures", "near-amend-genesis", "near-chain-configs", + "near-client", "near-dyn-configs", "near-jsonrpc-primitives", "near-mirror", diff --git a/chain/client/src/client.rs b/chain/client/src/client.rs index 9ac095da75a..64a029cb04e 100644 --- a/chain/client/src/client.rs +++ b/chain/client/src/client.rs @@ -26,7 +26,7 @@ use near_chain::{ BlockProcessingArtifact, BlockStatus, Chain, ChainGenesis, ChainStoreAccess, DoneApplyChunkCallback, Doomslug, DoomslugThresholdMode, Provenance, RuntimeAdapter, }; -use near_chain_configs::ClientConfig; +use near_chain_configs::{ClientConfig, UpdateableClientConfig}; use near_chunks::ShardsManager; use near_network::types::{ HighestHeightPeerInfo, NetworkRequests, PeerManagerAdapter, ReasonForBan, @@ -149,6 +149,12 @@ pub struct Client { flat_storage_creator: Option, } +impl Client { + pub(crate) fn update_client_config(&self, update_client_config: UpdateableClientConfig) { + self.config.expected_shutdown.update(update_client_config.expected_shutdown); + } +} + // Debug information about the upcoming block. #[derive(Default)] pub struct BlockDebugStatus { diff --git a/chain/client/src/client_actor.rs b/chain/client/src/client_actor.rs index cacf4d42350..d984185c41d 100644 --- a/chain/client/src/client_actor.rs +++ b/chain/client/src/client_actor.rs @@ -11,6 +11,7 @@ use crate::adapter::{ RecvPartialEncodedChunkRequest, RecvPartialEncodedChunkResponse, SetNetworkInfo, StateResponse, }; use crate::client::{Client, EPOCH_START_INFO_BLOCKS}; +use crate::config_updater::ConfigUpdater; use crate::debug::new_network_info_view; use crate::info::{display_sync_status, InfoHelper}; use crate::metrics::PARTIAL_ENCODED_CHUNK_RESPONSE_DELAY; @@ -38,7 +39,6 @@ use near_chunks::logic::cares_about_shard_this_or_next_epoch; use near_client_primitives::types::{ Error, GetNetworkInfo, NetworkInfoResponse, Status, StatusError, StatusSyncInfo, SyncStatus, }; -use near_dyn_configs::EXPECTED_SHUTDOWN_AT; #[cfg(feature = "test_features")] use near_network::types::NetworkAdversarialMessage; use near_network::types::ReasonForBan; @@ -69,7 +69,7 @@ use std::collections::HashMap; use std::sync::Arc; use std::thread; use std::time::{Duration, Instant}; -use tokio::sync::oneshot; +use tokio::sync::broadcast; use tracing::{debug, error, info, trace, warn}; /// Multiplier on `max_block_time` to wait until deciding that chain stalled. @@ -116,7 +116,10 @@ pub struct ClientActor { /// Synchronization measure to allow graceful shutdown. /// Informs the system when a ClientActor gets dropped. - shutdown_signal: Option>, + shutdown_signal: Option>, + + /// Manages updating the config. + config_updater: Option, } /// Blocks the program until given genesis time arrives. @@ -152,8 +155,9 @@ impl ClientActor { enable_doomslug: bool, rng_seed: RngSeed, ctx: &Context, - shutdown_signal: Option>, + shutdown_signal: Option>, adv: crate::adversarial::Controls, + config_updater: Option, ) -> Result { let state_parts_arbiter = Arbiter::new(); let self_addr = ctx.address(); @@ -222,7 +226,8 @@ impl ClientActor { #[cfg(feature = "sandbox")] fastforward_delta: 0, - shutdown_signal: shutdown_signal, + shutdown_signal, + config_updater, }) } } @@ -1129,14 +1134,20 @@ impl ClientActor { /// Returns the delay before the next time `check_triggers` should be called, which is /// min(time until the closest trigger, 1 second). fn check_triggers(&mut self, ctx: &mut Context) -> Duration { + if let Some(config_updater) = &mut self.config_updater { + config_updater.try_update(&|updateable_client_config| { + self.client.update_client_config(updateable_client_config) + }); + } + // Check block height to trigger expected shutdown if let Ok(head) = self.client.chain.head() { - let block_height_to_shutdown = - EXPECTED_SHUTDOWN_AT.load(std::sync::atomic::Ordering::Relaxed); - if block_height_to_shutdown > 0 && head.height >= block_height_to_shutdown { - info!(target: "client", "Expected shutdown triggered: head block({}) >= ({})", head.height, block_height_to_shutdown); - if let Some(tx) = self.shutdown_signal.take() { - let _ = tx.send(()); // Ignore send signal fail, it will send again in next trigger + if let Some(block_height_to_shutdown) = self.client.config.expected_shutdown.get() { + if head.height >= block_height_to_shutdown { + info!(target: "client", "Expected shutdown triggered: head block({}) >= ({:?})", head.height, block_height_to_shutdown); + if let Some(tx) = self.shutdown_signal.take() { + let _ = tx.send(()); // Ignore send signal fail, it will send again in next trigger + } } } } @@ -1755,7 +1766,12 @@ impl ClientActor { fn log_summary(&mut self) { let _span = tracing::debug_span!(target: "client", "log_summary").entered(); let _d = delay_detector::DelayDetector::new(|| "client log summary".into()); - self.info_helper.log_summary(&self.client, &self.node_id, &self.network_info) + self.info_helper.log_summary( + &self.client, + &self.node_id, + &self.network_info, + &self.config_updater, + ) } } @@ -1970,8 +1986,9 @@ pub fn start_client( network_adapter: Arc, validator_signer: Option>, telemetry_actor: Addr, - sender: Option>, + sender: Option>, adv: crate::adversarial::Controls, + config_updater: Option, ) -> (Addr, ArbiterHandle) { let client_arbiter = Arbiter::new(); let client_arbiter_handle = client_arbiter.handle(); @@ -1990,6 +2007,7 @@ pub fn start_client( ctx, sender, adv, + config_updater, ) .unwrap() }); diff --git a/chain/client/src/config_updater.rs b/chain/client/src/config_updater.rs new file mode 100644 index 00000000000..8f2389823cb --- /dev/null +++ b/chain/client/src/config_updater.rs @@ -0,0 +1,53 @@ +use near_chain_configs::UpdateableClientConfig; +use near_dyn_configs::{UpdateableConfigLoaderError, UpdateableConfigs}; +use std::sync::Arc; +use tokio::sync::broadcast::Receiver; + +#[derive(Debug)] +pub enum ClientConfigUpdateError {} + +/// Manages updating the config encapsulating. +pub struct ConfigUpdater { + /// Receives config updates while the node is running. + rx_config_update: Receiver>>, + + /// Represents the latest Error of reading the dynamically reloadable configs. + updateable_configs_error: Option>, +} + +impl ConfigUpdater { + pub fn new( + rx_config_update: Receiver>>, + ) -> Self { + Self { rx_config_update, updateable_configs_error: None } + } + + /// Check if any of the configs were updated. + /// If they did, the receiver (rx_config_update) will contain a clone of the new configs. + pub fn try_update(&mut self, update_client_config_fn: &dyn Fn(UpdateableClientConfig)) { + while let Ok(maybe_updateable_configs) = self.rx_config_update.try_recv() { + match maybe_updateable_configs { + Ok(updateable_configs) => { + if let Some(client_config) = updateable_configs.client_config { + update_client_config_fn(client_config); + tracing::info!(target: "config", "Updated ClientConfig"); + } + self.updateable_configs_error = None; + } + Err(err) => { + self.updateable_configs_error = Some(err.clone()); + } + } + } + } + + /// Prints an error if it's present. + pub fn report_status(&self) { + if let Some(updateable_configs_error) = &self.updateable_configs_error { + tracing::warn!( + target: "stats", + "Dynamically updateable configs are not valid. Please fix this ASAP otherwise the node will probably crash after restart: {}", + *updateable_configs_error); + } + } +} diff --git a/chain/client/src/info.rs b/chain/client/src/info.rs index d53d5bdfc8a..2251e99d840 100644 --- a/chain/client/src/info.rs +++ b/chain/client/src/info.rs @@ -1,3 +1,4 @@ +use crate::config_updater::ConfigUpdater; use crate::{metrics, rocksdb_metrics, SyncStatus}; use actix::Addr; use itertools::Itertools; @@ -125,6 +126,7 @@ impl InfoHelper { client: &crate::client::Client, node_id: &PeerId, network_info: &NetworkInfo, + config_updater: &Option, ) { let is_syncing = client.sync_status.is_syncing(); let head = unwrap_or_return!(client.chain.head()); @@ -190,6 +192,7 @@ impl InfoHelper { .unwrap_or(0), statistics, &client.config, + config_updater, ); self.log_chain_processing_info(client, &head.epoch_id); } @@ -206,6 +209,7 @@ impl InfoHelper { protocol_upgrade_block_height: BlockHeight, statistics: Option, client_config: &ClientConfig, + config_updater: &Option, ) { let use_colour = matches!(self.log_summary_style, LogSummaryStyle::Colored); let paint = |colour: ansi_term::Colour, text: Option| match text { @@ -268,12 +272,14 @@ impl InfoHelper { paint(ansi_term::Colour::Blue, machine_info_log), ); if catchup_status_log != "" { - info!(target:"stats", "Catchups\n{}", catchup_status_log); + info!(target: "stats", "Catchups\n{}", catchup_status_log); } if let Some(statistics) = statistics { rocksdb_metrics::export_stats_as_metrics(statistics); } - + if let Some(config_updater) = &config_updater { + config_updater.report_status(); + } let (cpu_usage, memory_usage) = proc_info.unwrap_or_default(); let is_validator = validator_info.map(|v| v.is_validator).unwrap_or_default(); (metrics::IS_VALIDATOR.set(is_validator as i64)); diff --git a/chain/client/src/lib.rs b/chain/client/src/lib.rs index ee8aaa224ad..e8451dd3de5 100644 --- a/chain/client/src/lib.rs +++ b/chain/client/src/lib.rs @@ -14,12 +14,14 @@ pub use crate::adapter::{ }; pub use crate::client::Client; pub use crate::client_actor::{start_client, ClientActor}; +pub use crate::config_updater::ConfigUpdater; pub use crate::view_client::{start_view_client, ViewClientActor}; pub mod adapter; pub mod adversarial; mod client; mod client_actor; +mod config_updater; pub mod debug; mod info; mod metrics; diff --git a/chain/client/src/test_utils.rs b/chain/client/src/test_utils.rs index eddb0f0b8dc..d33d5e26557 100644 --- a/chain/client/src/test_utils.rs +++ b/chain/client/src/test_utils.rs @@ -265,6 +265,7 @@ pub fn setup( ctx, None, adv, + None, ) .unwrap(); (genesis_block, client, view_client_addr) diff --git a/core/chain-configs/Cargo.toml b/core/chain-configs/Cargo.toml index dcc9fb90835..59347e863d0 100644 --- a/core/chain-configs/Cargo.toml +++ b/core/chain-configs/Cargo.toml @@ -15,6 +15,7 @@ anyhow.workspace = true chrono.workspace = true derive_more.workspace = true num-rational.workspace = true +once_cell.workspace = true serde.workspace = true serde_json.workspace = true sha2.workspace = true @@ -22,6 +23,7 @@ smart-default.workspace = true tracing.workspace = true near-crypto = { path = "../crypto" } +near-o11y = { path = "../o11y" } near-primitives = { path = "../primitives" } [features] diff --git a/core/chain-configs/src/client_config.rs b/core/chain-configs/src/client_config.rs index ad25750d3a5..810ea0332dc 100644 --- a/core/chain-configs/src/client_config.rs +++ b/core/chain-configs/src/client_config.rs @@ -5,7 +5,10 @@ use std::time::Duration; use serde::{Deserialize, Serialize}; -use near_primitives::types::{AccountId, BlockHeightDelta, Gas, NumBlocks, NumSeats, ShardId}; +use crate::MutableConfigValue; +use near_primitives::types::{ + AccountId, BlockHeight, BlockHeightDelta, Gas, NumBlocks, NumSeats, ShardId, +}; use near_primitives::version::Version; pub const TEST_STATE_SYNC_TIMEOUT: u64 = 5; @@ -70,7 +73,8 @@ impl GCConfig { } } -#[derive(Clone, Serialize, Deserialize)] +/// ClientConfig where some fields can be updated at runtime. +#[derive(Clone)] pub struct ClientConfig { /// Version of the binary. pub version: Version, @@ -78,6 +82,8 @@ pub struct ClientConfig { pub chain_id: String, /// Listening rpc port for status. pub rpc_addr: Option, + /// Graceful shutdown at expected block height. + pub expected_shutdown: MutableConfigValue>, /// Duration to check for producing / skipping block. pub block_production_tracking_delay: Duration, /// Minimum duration before producing block. @@ -182,10 +188,11 @@ impl ClientConfig { because non-archival nodes must save trie changes in order to do do garbage collection." ); - ClientConfig { + Self { version: Default::default(), chain_id: "unittest".to_string(), rpc_addr: Some("0.0.0.0:3030".to_string()), + expected_shutdown: MutableConfigValue::new(None, "expected_shutdown"), block_production_tracking_delay: Duration::from_millis(std::cmp::max( 10, min_block_prod_time / 5, diff --git a/core/chain-configs/src/lib.rs b/core/chain-configs/src/lib.rs index ab2fcb53ca7..77925f77c37 100644 --- a/core/chain-configs/src/lib.rs +++ b/core/chain-configs/src/lib.rs @@ -1,6 +1,8 @@ mod client_config; mod genesis_config; pub mod genesis_validate; +mod metrics; +mod updateable_config; pub use client_config::{ ClientConfig, GCConfig, LogSummaryStyle, DEFAULT_GC_NUM_EPOCHS_TO_KEEP, @@ -10,3 +12,4 @@ pub use genesis_config::{ get_initial_supply, stream_records_from_file, Genesis, GenesisChangeConfig, GenesisConfig, GenesisRecords, GenesisValidationMode, ProtocolConfig, ProtocolConfigView, }; +pub use updateable_config::{MutableConfigValue, UpdateableClientConfig}; diff --git a/core/chain-configs/src/metrics.rs b/core/chain-configs/src/metrics.rs new file mode 100644 index 00000000000..926a893dc12 --- /dev/null +++ b/core/chain-configs/src/metrics.rs @@ -0,0 +1,11 @@ +use near_o11y::metrics::{try_create_int_gauge_vec, IntGaugeVec}; +use once_cell::sync::Lazy; + +pub static CONFIG_MUTABLE_FIELD: Lazy = Lazy::new(|| { + try_create_int_gauge_vec( + "near_config_mutable_field", + "Timestamp and value of a mutable config field", + &["field_name", "timestamp", "value"], + ) + .unwrap() +}); diff --git a/core/chain-configs/src/updateable_config.rs b/core/chain-configs/src/updateable_config.rs new file mode 100644 index 00000000000..dc2c97b5ffb --- /dev/null +++ b/core/chain-configs/src/updateable_config.rs @@ -0,0 +1,75 @@ +use crate::metrics; +use chrono::{DateTime, Utc}; +use near_primitives::time::Clock; +use near_primitives::types::BlockHeight; +use serde::{Deserialize, Serialize}; +use std::fmt::Debug; +use std::sync::{Arc, Mutex}; + +/// A wrapper for a config value that can be updated while the node is running. +/// When initializing sub-objects (e.g. `ShardsManager`), please make sure to +/// pass this wrapper instead of passing a value from a single moment in time. +/// See `expected_shutdown` for an example how to use it. +#[derive(Clone)] +pub struct MutableConfigValue { + value: Arc>, + // For metrics. + // Mutable config values are exported to prometheus with labels [field_name][last_update][value]. + field_name: String, + // For metrics. + // Mutable config values are exported to prometheus with labels [field_name][last_update][value]. + last_update: DateTime, +} + +impl MutableConfigValue { + /// Initializes a value. + /// `field_name` is needed to export the config value as a prometheus metric. + pub fn new(val: T, field_name: &str) -> Self { + let res = Self { + value: Arc::new(Mutex::new(val)), + field_name: field_name.to_string(), + last_update: Clock::utc(), + }; + res.set_metric_value(val, 1); + res + } + + pub fn get(&self) -> T { + *self.value.lock().unwrap() + } + + pub fn update(&self, val: T) { + let mut lock = self.value.lock().unwrap(); + if *lock != val { + tracing::info!(target: "config", "Updated config field '{}' from {:?} to {:?}", self.field_name, *lock, val); + self.set_metric_value(*lock, 0); + *lock = val; + self.set_metric_value(val, 1); + } else { + tracing::info!(target: "config", "Mutable config field '{}' remains the same: {:?}", self.field_name, val); + } + } + + fn set_metric_value(&self, value: T, metric_value: i64) { + // Use field_name as a label to tell different mutable config values apart. + // Use timestamp as a label to give some idea to the node operator (or + // people helping them debug their node) when exactly and what values + // exactly were part of the config. + // Use the config value as a label to make this work with config values + // of any type: int, float, string or even a composite object. + metrics::CONFIG_MUTABLE_FIELD + .with_label_values(&[ + &self.field_name, + &self.last_update.timestamp().to_string(), + &format!("{:?}", value), + ]) + .set(metric_value); + } +} + +#[derive(Default, Clone, Serialize, Deserialize)] +/// A subset of Config that can be updated white the node is running. +pub struct UpdateableClientConfig { + /// Graceful shutdown at expected block height. + pub expected_shutdown: Option, +} diff --git a/core/dyn-configs/Cargo.toml b/core/dyn-configs/Cargo.toml index 715ff42478b..cc0876aa9cc 100644 --- a/core/dyn-configs/Cargo.toml +++ b/core/dyn-configs/Cargo.toml @@ -11,6 +11,15 @@ repository = "https://github.com/near/nearcore" description = "Dynamic configure helpers for the near codebase" [dependencies] +anyhow.workspace = true once_cell.workspace = true prometheus.workspace = true +serde.workspace = true +serde_json.workspace = true +thiserror.workspace = true +tokio.workspace = true +tracing.workspace = true + +near-chain-configs = { path = "../chain-configs" } near-o11y = { path = "../o11y" } +near-primitives = { path = "../primitives" } diff --git a/core/dyn-configs/README.md b/core/dyn-configs/README.md index b6ba607094f..b996d7441e9 100644 --- a/core/dyn-configs/README.md +++ b/core/dyn-configs/README.md @@ -1,5 +1,26 @@ -Dynamic config helpers for the NEAR codebase. +Dynamic config helpers for the NEAR codebase. -This crate contains all utilities to dynamic control neard. +This crate contains utilities that allow to reconfigure the node while it is running. -- `EXPECTED_SHUTDOWN_AT`: the specified block height neard will gracefully shutdown at. +## How to: + +### Logging and tracing + +Make changes to `log_config.json` and send `SIGHUP` signal to the `neard` process. + +### Other config values + +Makes changes to `config.json` and send `SIGHUP` signal to the `neard` process. + +#### Fields of config that can be changed while the node is running: + +- `expected_shutdown`: the specified block height neard will gracefully shutdown at. + +#### Changing other fields of `config.json` + +The changes to other fields of `config.json` will be silently ignored as long as +`config.json` remains a valid json object and passes internal validation. + +Please be careful about making changes to `config.json` because when a node +starts (or restarts), it checks the validity of the config files and crashes if +detects any issues. diff --git a/core/dyn-configs/src/lib.rs b/core/dyn-configs/src/lib.rs index 4ad7cc75e0e..d0126380cc3 100644 --- a/core/dyn-configs/src/lib.rs +++ b/core/dyn-configs/src/lib.rs @@ -1,28 +1,73 @@ #![doc = include_str!("../README.md")] -use near_o11y::metrics::{try_create_int_counter, IntCounter}; -use once_cell::sync::Lazy; -use std::sync::atomic::{AtomicU64, Ordering}; - -/// An indicator for dynamic config changes -pub static DYN_CONFIG_CHANGE: Lazy = Lazy::new(|| { - try_create_int_counter( - "near_dynamic_config_changes", - "Total number of dynamic configuration changes", - ) - .unwrap() -}); - -// NOTE: AtomicU64 is the same unit as BlockHeight, and use to store the expected blockheight to -// shutdown -pub static EXPECTED_SHUTDOWN_AT: AtomicU64 = AtomicU64::new(0); - -/// Reload the dynamic config, and increase the counting metric near_dynamic_config_changes -pub fn reload(expected_shutdown: Option) { - if let Some(expected_shutdown) = expected_shutdown { - EXPECTED_SHUTDOWN_AT.store(expected_shutdown, Ordering::Relaxed); - } else { - EXPECTED_SHUTDOWN_AT.store(0, Ordering::Relaxed); +use near_chain_configs::UpdateableClientConfig; +use near_o11y::log_config::LogConfig; +use near_primitives::time::Clock; +use serde::{Deserialize, Serialize}; +use std::path::PathBuf; +use std::sync::Arc; +use tokio::sync::broadcast::Sender; + +mod metrics; + +#[derive(Serialize, Deserialize, Clone, Default)] +/// Contains the latest state of configs which can be updated at runtime. +pub struct UpdateableConfigs { + /// Contents of the file LOG_CONFIG_FILENAME. + pub log_config: Option, + /// Contents of the `config.json` corresponding to the mutable fields of `ClientConfig`. + pub client_config: Option, +} + +/// Pushes the updates to listeners. +#[derive(Default)] +pub struct UpdateableConfigLoader { + /// Notifies receivers about the new config values available. + tx: Option>>>, +} + +#[derive(thiserror::Error, Debug)] +#[non_exhaustive] +pub enum UpdateableConfigLoaderError { + #[error("Failed to parse a dynamic config file {file:?}: {err:?}")] + Parse { file: PathBuf, err: serde_json::Error }, + #[error("Can't open or read a dynamic config file {file:?}: {err:?}")] + OpenAndRead { file: PathBuf, err: std::io::Error }, + #[error("Can't open or read the config file {file:?}: {err:?}")] + ConfigFileError { file: PathBuf, err: anyhow::Error }, + #[error("One or multiple dynamic config files reload errors {0:?}")] + Errors(Vec), + #[error("No home dir set")] + NoHomeDir(), +} + +impl UpdateableConfigLoader { + pub fn new( + updateable_configs: UpdateableConfigs, + tx: Sender>>, + ) -> Self { + let mut result = Self { tx: Some(tx) }; + result.reload(Ok(updateable_configs)); + result + } + + pub fn reload( + &mut self, + updateable_configs: Result, + ) { + match updateable_configs { + Ok(updateable_configs) => { + self.tx.as_ref().map(|tx| tx.send(Ok(updateable_configs.clone()))); + Self::update_metrics(); + } + Err(err) => { + self.tx.as_ref().map(|tx| tx.send(Err(Arc::new(err)))); + } + } + } + + fn update_metrics() { + metrics::CONFIG_RELOAD_TIMESTAMP.set(Clock::utc().timestamp()); + metrics::CONFIG_RELOADS.inc(); } - DYN_CONFIG_CHANGE.inc(); } diff --git a/core/dyn-configs/src/metrics.rs b/core/dyn-configs/src/metrics.rs new file mode 100644 index 00000000000..55442c3d230 --- /dev/null +++ b/core/dyn-configs/src/metrics.rs @@ -0,0 +1,18 @@ +use near_o11y::metrics::{try_create_int_counter, try_create_int_gauge, IntCounter, IntGauge}; +use once_cell::sync::Lazy; + +pub static CONFIG_RELOADS: Lazy = Lazy::new(|| { + try_create_int_counter( + "near_config_reloads_total", + "Number of times the configs were reloaded during the current run of the process", + ) + .unwrap() +}); + +pub static CONFIG_RELOAD_TIMESTAMP: Lazy = Lazy::new(|| { + try_create_int_gauge( + "near_config_reload_timestamp_seconds", + "Timestamp of the last reload of the config", + ) + .unwrap() +}); diff --git a/core/o11y/src/lib.rs b/core/o11y/src/lib.rs index 7674237de0d..a678a3314c4 100644 --- a/core/o11y/src/lib.rs +++ b/core/o11y/src/lib.rs @@ -27,6 +27,7 @@ use tracing_subscriber::{fmt, reload, EnvFilter, Layer, Registry}; /// Custom tracing subscriber implementation that produces IO traces. pub mod context; mod io_tracer; +pub mod log_config; pub mod macros; pub mod metrics; pub mod pretty; @@ -80,6 +81,7 @@ static DEFAULT_OTLP_LEVEL: OnceCell = OnceCell::new(); /// The default value for the `RUST_LOG` environment variable if one isn't specified otherwise. pub const DEFAULT_RUST_LOG: &str = "tokio_reactor=info,\ + config=info,\ near=info,\ recompress=info,\ stats=info,\ @@ -358,6 +360,12 @@ pub fn default_subscriber( } } +pub fn set_default_otlp_level(options: &Options) { + // Record the initial tracing level specified as a command-line flag. Use this recorded value to + // reset opentelemetry filter when the LogConfig file gets deleted. + DEFAULT_OTLP_LEVEL.set(options.opentelemetry).unwrap(); +} + /// Constructs a subscriber set to the option appropriate for the NEAR code. /// /// The subscriber enables logging, tracing and io tracing. @@ -378,9 +386,7 @@ pub async fn default_subscriber_with_opentelemetry( let subscriber = tracing_subscriber::registry(); - // Record the initial tracing level specified as a command-line flag. Use this recorded value to - // reset opentelemetry filter when the LogConfig file gets deleted. - DEFAULT_OTLP_LEVEL.set(options.opentelemetry).unwrap(); + set_default_otlp_level(options); let (subscriber, handle) = add_non_blocking_log_layer( env_filter, @@ -440,6 +446,20 @@ pub enum ReloadError { Parse(#[source] BuildEnvFilterError), } +pub fn reload_log_config(config: Option<&log_config::LogConfig>) -> Result<(), Vec> { + if let Some(config) = config { + reload( + config.rust_log.as_ref().map(|s| s.as_str()), + config.verbose_module.as_ref().map(|s| s.as_str()), + config.opentelemetry_level, + ) + } else { + // When the LOG_CONFIG_FILENAME is not available, reset to the tracing and logging config + // when the node was started. + reload(None, None, None) + } +} + /// Constructs new filters for the logging and opentelemetry layers. /// /// Attempts to reload all available errors. Returns errors for each layer that failed to reload. @@ -457,8 +477,10 @@ pub fn reload( let log_reload_result = LOG_LAYER_RELOAD_HANDLE.get().map_or( Err(ReloadError::NoLogReloadHandle), |reload_handle| { - let mut builder = - rust_log.map_or_else(EnvFilterBuilder::from_env, EnvFilterBuilder::new); + let mut builder = rust_log.map_or_else( + || EnvFilterBuilder::from_env(), + |rust_log| EnvFilterBuilder::new(rust_log), + ); if let Some(module) = verbose_module { builder = builder.verbose(Some(module)); } diff --git a/core/o11y/src/log_config.rs b/core/o11y/src/log_config.rs new file mode 100644 index 00000000000..46845bffe63 --- /dev/null +++ b/core/o11y/src/log_config.rs @@ -0,0 +1,13 @@ +use serde::{Deserialize, Serialize}; + +/// Configures logging. +#[derive(Default, Serialize, Deserialize, Clone, Debug)] +pub struct LogConfig { + /// Comma-separated list of EnvFitler directives. + pub rust_log: Option, + /// Some("") enables global debug logging. + /// Some("module") enables debug logging for "module". + pub verbose_module: Option, + /// Verbosity level of collected traces. + pub opentelemetry_level: Option, +} diff --git a/integration-tests/src/tests/network/runner.rs b/integration-tests/src/tests/network/runner.rs index 622948edcff..a88a0e74a3e 100644 --- a/integration-tests/src/tests/network/runner.rs +++ b/integration-tests/src/tests/network/runner.rs @@ -76,6 +76,7 @@ fn setup_network_node( telemetry_actor, None, adv.clone(), + None, ) .0; let view_client_actor = start_view_client( diff --git a/nearcore/src/config.rs b/nearcore/src/config.rs index f1a6db1f4f5..32966310e2b 100644 --- a/nearcore/src/config.rs +++ b/nearcore/src/config.rs @@ -17,7 +17,7 @@ use tracing::{info, warn}; use near_chain_configs::{ get_initial_supply, ClientConfig, GCConfig, Genesis, GenesisConfig, GenesisValidationMode, - LogSummaryStyle, + LogSummaryStyle, MutableConfigValue, }; use near_crypto::{InMemorySigner, KeyFile, KeyType, PublicKey, Signer}; #[cfg(feature = "json_rpc")] @@ -31,8 +31,8 @@ use near_primitives::shard_layout::account_id_to_shard_id; use near_primitives::shard_layout::ShardLayout; use near_primitives::state_record::StateRecord; use near_primitives::types::{ - AccountId, AccountInfo, Balance, BlockHeightDelta, EpochHeight, Gas, NumBlocks, NumSeats, - NumShards, ShardId, + AccountId, AccountInfo, Balance, BlockHeight, BlockHeightDelta, EpochHeight, Gas, NumBlocks, + NumSeats, NumShards, ShardId, }; use near_primitives::utils::{generate_random_string, get_num_seats_per_shard}; use near_primitives::validator_signer::{InMemoryValidatorSigner, ValidatorSigner}; @@ -201,6 +201,12 @@ fn default_trie_viewer_state_size_limit() -> Option { Some(50_000) } +#[derive(thiserror::Error, Debug)] +pub enum ConfigValidationError { + #[error("Configuration with archive = false and save_trie_changes = false is not supported because non-archival nodes must save trie changes in order to do do garbage collection.")] + TrieChanges, +} + #[derive(Serialize, Deserialize, Clone, Debug)] pub struct Consensus { /// Minimum number of peers to start syncing. @@ -338,6 +344,8 @@ pub struct Config { /// Deprecated; use `store.migration_snapshot` instead. #[serde(skip_serializing_if = "Option::is_none")] pub db_migration_snapshot_path: Option, + #[serde(default, skip_serializing_if = "Option::is_none")] + pub expected_shutdown: Option, } fn is_false(value: &bool) -> bool { @@ -374,6 +382,7 @@ impl Default for Config { store: near_store::StoreConfig::default(), #[cfg(feature = "cold_store")] cold_store: None, + expected_shutdown: None, } } } @@ -411,15 +420,23 @@ impl Config { ); } - assert!( - config.archive || config.save_trie_changes, - "Configuration with archive = false and save_trie_changes = false is not supported \ - because non-archival nodes must save trie changes in order to do do garbage collection." - ); - + config.validate()?; Ok(config) } + /// Does semantic config validation. + /// This is the place to check that all config values make sense and fit well together. + /// `validate()` is called every time `config.json` is read. + fn validate(&self) -> Result<(), ConfigValidationError> { + if self.archive == self.save_trie_changes { + Err(ConfigValidationError::TrieChanges) + } else { + Ok(()) + } + // TODO: Add more config validation. + // TODO: Validate `ClientConfig` instead. + } + pub fn write_to_file(&self, path: &Path) -> std::io::Result<()> { let mut file = File::create(path)?; let str = serde_json::to_string_pretty(self)?; @@ -571,6 +588,10 @@ impl NearConfig { version: Default::default(), chain_id: genesis.config.chain_id.clone(), rpc_addr: config.rpc_addr().map(|addr| addr.to_owned()), + expected_shutdown: MutableConfigValue::new( + config.expected_shutdown, + "expected_shutdown", + ), block_production_tracking_delay: config.consensus.block_production_tracking_delay, min_block_production_delay: config.consensus.min_block_production_delay, max_block_production_delay: config.consensus.max_block_production_delay, diff --git a/nearcore/src/dyn_config.rs b/nearcore/src/dyn_config.rs new file mode 100644 index 00000000000..e68fd67b3a6 --- /dev/null +++ b/nearcore/src/dyn_config.rs @@ -0,0 +1,77 @@ +use crate::config::Config; +use near_chain_configs::UpdateableClientConfig; +use near_dyn_configs::{UpdateableConfigLoaderError, UpdateableConfigs}; +use near_o11y::log_config::LogConfig; +use serde::Deserialize; +use std::path::{Path, PathBuf}; + +const LOG_CONFIG_FILENAME: &str = "log_config.json"; + +/// This function gets called at the startup and each time a config needs to be reloaded. +pub fn read_updateable_configs( + home_dir: &Path, +) -> Result { + let mut errs = vec![]; + let log_config = match read_log_config(home_dir) { + Ok(config) => config, + Err(err) => { + errs.push(err); + None + } + }; + let updateable_client_config = + match Config::from_file(&home_dir.join(crate::config::CONFIG_FILENAME)) + .map(get_updateable_client_config) + { + Ok(config) => Some(config), + Err(err) => { + errs.push(UpdateableConfigLoaderError::ConfigFileError { + file: PathBuf::from(crate::config::CONFIG_FILENAME), + err, + }); + None + } + }; + if errs.is_empty() { + crate::metrics::CONFIG_CORRECT.set(1); + Ok(UpdateableConfigs { log_config, client_config: updateable_client_config }) + } else { + tracing::warn!(target: "neard", "Dynamically updateable configs are not valid. Please fix this ASAP otherwise the node will be unable to restart: {:?}", &errs); + crate::metrics::CONFIG_CORRECT.set(0); + Err(UpdateableConfigLoaderError::Errors(errs)) + } +} + +pub fn get_updateable_client_config(config: Config) -> UpdateableClientConfig { + // All fields that can be updated while the node is running should be explicitly set here. + // Keep this list in-sync with `core/dyn-configs/README.md`. + UpdateableClientConfig { expected_shutdown: config.expected_shutdown } +} + +fn read_log_config(home_dir: &Path) -> Result, UpdateableConfigLoaderError> { + read_json_config::(&home_dir.join(LOG_CONFIG_FILENAME)) +} + +fn read_json_config( + path: &Path, +) -> Result, UpdateableConfigLoaderError> +where + for<'a> T: Deserialize<'a>, +{ + match std::fs::read_to_string(path) { + Ok(config_str) => match serde_json::from_str::(&config_str) { + Ok(config) => { + tracing::info!(target: "neard", config=?config, "Changing the config {path:?}."); + return Ok(Some(config)); + } + Err(err) => Err(UpdateableConfigLoaderError::Parse { file: path.to_path_buf(), err }), + }, + Err(err) => match err.kind() { + std::io::ErrorKind::NotFound => { + tracing::info!(target: "neard", ?err, "Reset the config {path:?} because the config file doesn't exist."); + return Ok(None); + } + _ => Err(UpdateableConfigLoaderError::OpenAndRead { file: path.to_path_buf(), err }), + }, + } +} diff --git a/nearcore/src/lib.rs b/nearcore/src/lib.rs index 3ce9d2f0da3..1cedbd285e2 100644 --- a/nearcore/src/lib.rs +++ b/nearcore/src/lib.rs @@ -6,7 +6,7 @@ use actix_rt::ArbiterHandle; use actix_web; use anyhow::Context; use near_chain::{Chain, ChainGenesis}; -use near_client::{start_client, start_view_client, ClientActor, ViewClientActor}; +use near_client::{start_client, start_view_client, ClientActor, ConfigUpdater, ViewClientActor}; use near_network::time; use near_network::types::NetworkRecipient; use near_network::PeerManagerActor; @@ -15,12 +15,13 @@ use near_store::{DBCol, Mode, NodeStorage, StoreOpenerError, Temperature}; use near_telemetry::TelemetryActor; use std::path::{Path, PathBuf}; use std::sync::Arc; -use tokio::sync::oneshot; +use tokio::sync::broadcast; use tracing::{info, trace}; pub mod append_only_map; pub mod config; mod download_file; +pub mod dyn_config; mod metrics; pub mod migrations; mod runtime; @@ -153,7 +154,7 @@ pub struct NearNode { } pub fn start_with_config(home_dir: &Path, config: NearConfig) -> anyhow::Result { - start_with_config_and_synchronization(home_dir, config, None) + start_with_config_and_synchronization(home_dir, config, None, None) } pub fn start_with_config_and_synchronization( @@ -161,7 +162,8 @@ pub fn start_with_config_and_synchronization( mut config: NearConfig, // 'shutdown_signal' will notify the corresponding `oneshot::Receiver` when an instance of // `ClientActor` gets dropped. - shutdown_signal: Option>, + shutdown_signal: Option>, + config_updater: Option, ) -> anyhow::Result { let store = open_storage(home_dir, &mut config)?; @@ -199,8 +201,9 @@ pub fn start_with_config_and_synchronization( network_adapter.clone(), config.validator_signer, telemetry, - shutdown_signal, + shutdown_signal.clone(), adv, + config_updater, ); #[allow(unused_mut)] diff --git a/nearcore/src/metrics.rs b/nearcore/src/metrics.rs index 4304119b0e4..002b2bdf2b1 100644 --- a/nearcore/src/metrics.rs +++ b/nearcore/src/metrics.rs @@ -1,4 +1,6 @@ -use near_o11y::metrics::{linear_buckets, try_create_histogram_vec, HistogramVec}; +use near_o11y::metrics::{ + linear_buckets, try_create_histogram_vec, try_create_int_gauge, HistogramVec, IntGauge, +}; use once_cell::sync::Lazy; pub static APPLY_CHUNK_DELAY: Lazy = Lazy::new(|| { @@ -8,7 +10,7 @@ pub static APPLY_CHUNK_DELAY: Lazy = Lazy::new(|| { &["tgas_ceiling"], Some(linear_buckets(0.0, 0.05, 50).unwrap()), ) - .unwrap() + .unwrap() }); pub static SECONDS_PER_PETAGAS: Lazy = Lazy::new(|| { @@ -24,3 +26,11 @@ pub static SECONDS_PER_PETAGAS: Lazy = Lazy::new(|| { ) .unwrap() }); + +pub(crate) static CONFIG_CORRECT: Lazy = Lazy::new(|| { + try_create_int_gauge( + "near_config_correct", + "Are the current dynamically loadable configs correct", + ) + .unwrap() +}); diff --git a/neard/Cargo.toml b/neard/Cargo.toml index bc22bf428e7..b9b36383265 100644 --- a/neard/Cargo.toml +++ b/neard/Cargo.toml @@ -32,6 +32,7 @@ tracing.workspace = true nearcore = { path = "../nearcore" } near-amend-genesis = { path = "../tools/amend-genesis" } near-chain-configs = { path = "../core/chain-configs" } +near-client = { path = "../chain/client" } near-cold-store-tool = { path = "../tools/cold-store", package = "cold-store-tool" } near-dyn-configs = { path = "../core/dyn-configs" } near-jsonrpc-primitives = { path = "../chain/jsonrpc-primitives" } diff --git a/neard/src/cli.rs b/neard/src/cli.rs index e38753ed117..75e6c4b12c9 100644 --- a/neard/src/cli.rs +++ b/neard/src/cli.rs @@ -1,20 +1,18 @@ #[cfg(unix)] -use crate::watchers::Watcher; -use crate::watchers::{ - dyn_config_watcher::DynConfig, log_config_watcher::LogConfig, UpdateBehavior, -}; use anyhow::Context; use clap::{Args, Parser}; use near_amend_genesis::AmendGenesisCommand; use near_chain_configs::GenesisValidationMode; +use near_client::ConfigUpdater; #[cfg(feature = "cold_store")] use near_cold_store_tool::ColdStoreCommand; +use near_dyn_configs::{UpdateableConfigLoader, UpdateableConfigLoaderError, UpdateableConfigs}; use near_jsonrpc_primitives::types::light_client::RpcLightClientExecutionProofResponse; use near_mirror::MirrorCommand; use near_o11y::tracing_subscriber::EnvFilter; use near_o11y::{ default_subscriber, default_subscriber_with_opentelemetry, BuildEnvFilterError, - EnvFilterBuilder, OpenTelemetryLevel, + EnvFilterBuilder, }; use near_ping::PingCommand; use near_primitives::hash::CryptoHash; @@ -29,8 +27,9 @@ use std::fs::File; use std::io::BufReader; use std::net::SocketAddr; use std::path::{Path, PathBuf}; -use tokio::sync::oneshot; -use tokio::sync::oneshot::Receiver; +use std::sync::Arc; +use tokio::sync::broadcast; +use tokio::sync::broadcast::Receiver; use tracing::{debug, error, info, warn}; /// NEAR Protocol Node @@ -457,7 +456,9 @@ impl RunCmd { } } - let (tx, rx) = oneshot::channel::<()>(); + let (tx_crash, mut rx_crash) = broadcast::channel::<()>(16); + let (tx_config_update, rx_config_update) = + broadcast::channel::>>(16); let sys = actix::System::new(); sys.block_on(async move { @@ -476,11 +477,31 @@ impl RunCmd { .await .global(); - let nearcore::NearNode { rpc_servers, .. } = - nearcore::start_with_config_and_synchronization(home_dir, near_config, Some(tx)) - .expect("start_with_config"); + let updateable_configs = nearcore::dyn_config::read_updateable_configs(home_dir) + .unwrap_or_else(|e| panic!("Error reading dynamic configs: {:#}", e)); + let mut updateable_config_loader = + UpdateableConfigLoader::new(updateable_configs.clone(), tx_config_update); + let config_updater = ConfigUpdater::new(rx_config_update); - let sig = wait_for_interrupt_signal(home_dir, rx).await; + let nearcore::NearNode { rpc_servers, .. } = + nearcore::start_with_config_and_synchronization( + home_dir, + near_config, + Some(tx_crash), + Some(config_updater), + ) + .expect("start_with_config"); + + let sig = loop { + let sig = wait_for_interrupt_signal(home_dir, &mut rx_crash).await; + if sig == "SIGHUP" { + let maybe_updateable_configs = + nearcore::dyn_config::read_updateable_configs(home_dir); + updateable_config_loader.reload(maybe_updateable_configs); + } else { + break sig; + } + }; warn!(target: "neard", "{}, stopping... this may take a few minutes.", sig); futures::future::join_all(rpc_servers.iter().map(|(name, server)| async move { server.stop(true).await; @@ -488,9 +509,9 @@ impl RunCmd { })) .await; actix::System::current().stop(); - // Disable the subscriber to properly shutdown the tracer. - near_o11y::reload(Some("error"), None, Some(OpenTelemetryLevel::OFF)).unwrap(); + near_o11y::reload(Some("error"), None, Some(near_o11y::OpenTelemetryLevel::OFF)) + .unwrap(); }); sys.run().unwrap(); info!(target: "neard", "Waiting for RocksDB to gracefully shutdown"); @@ -499,38 +520,26 @@ impl RunCmd { } #[cfg(not(unix))] -async fn wait_for_interrupt_signal(_home_dir: &Path, mut _rx_crash: Receiver<()>) -> &str { +async fn wait_for_interrupt_signal(_home_dir: &Path, mut _rx_crash: &Receiver<()>) -> &str { // TODO(#6372): Support graceful shutdown on windows. tokio::signal::ctrl_c().await.unwrap(); "Ctrl+C" } #[cfg(unix)] -fn update_watchers(home_dir: &Path, behavior: UpdateBehavior) { - LogConfig::update(home_dir.join("log_config.json"), &behavior); - DynConfig::update(home_dir.join("dyn_config.json"), &behavior); -} - -#[cfg(unix)] -async fn wait_for_interrupt_signal(home_dir: &Path, mut rx_crash: Receiver<()>) -> &str { - // Apply all watcher config file if it exists. - update_watchers(&home_dir, UpdateBehavior::UpdateOnlyIfExists); - +async fn wait_for_interrupt_signal(_home_dir: &Path, rx_crash: &mut Receiver<()>) -> &'static str { use tokio::signal::unix::{signal, SignalKind}; let mut sigint = signal(SignalKind::interrupt()).unwrap(); let mut sigterm = signal(SignalKind::terminate()).unwrap(); let mut sighup = signal(SignalKind::hangup()).unwrap(); - loop { - break tokio::select! { - _ = sigint.recv() => "SIGINT", - _ = sigterm.recv() => "SIGTERM", - _ = sighup.recv() => { - update_watchers(&home_dir, UpdateBehavior::UpdateOrReset); - continue; - }, - _ = &mut rx_crash => "ClientActor died", - }; + tokio::select! { + _ = sigint.recv() => "SIGINT", + _ = sigterm.recv() => "SIGTERM", + _ = sighup.recv() => { + "SIGHUP" + }, + _ = rx_crash.recv() => "ClientActor died", } } @@ -795,7 +804,7 @@ mod tests { ) ); - // Proof with a wroing outcome (as user specified wrong shard). + // Proof with a wrong outcome (as user specified wrong shard). assert_eq!( VerifyProofSubCommand::verify_json( serde_json::from_slice(include_bytes!("../res/invalid_proof.json")).unwrap() diff --git a/neard/src/main.rs b/neard/src/main.rs index ab11ffac7c0..d322b6066a6 100644 --- a/neard/src/main.rs +++ b/neard/src/main.rs @@ -1,5 +1,4 @@ mod cli; -mod watchers; use self::cli::NeardCmd; use anyhow::Context; diff --git a/neard/src/watchers/dyn_config_watcher.rs b/neard/src/watchers/dyn_config_watcher.rs deleted file mode 100644 index 81ebeba1fb4..00000000000 --- a/neard/src/watchers/dyn_config_watcher.rs +++ /dev/null @@ -1,22 +0,0 @@ -use crate::watchers::{WatchConfigError, Watcher}; -use near_dyn_configs::reload; -use serde::{Deserialize, Serialize}; - -/// Configures logging. -#[derive(Serialize, Deserialize, Clone, Debug)] -pub(crate) struct DynConfig { - /// Graceful shutdown at expected blockheight - pub expected_shutdown: Option, -} - -impl Watcher for DynConfig { - fn reload(config: Option) -> Result<(), WatchConfigError> { - if let Some(config) = config { - reload(config.expected_shutdown); - Ok(()) - } else { - reload(None); - Ok(()) - } - } -} diff --git a/neard/src/watchers/log_config_watcher.rs b/neard/src/watchers/log_config_watcher.rs deleted file mode 100644 index 9824b98a17b..00000000000 --- a/neard/src/watchers/log_config_watcher.rs +++ /dev/null @@ -1,31 +0,0 @@ -use crate::watchers::{WatchConfigError, Watcher}; -use near_o11y::{reload, OpenTelemetryLevel, ReloadError}; -use serde::{Deserialize, Serialize}; - -/// Configures logging. -#[derive(Default, Serialize, Deserialize, Clone, Debug)] -pub(crate) struct LogConfig { - /// Comma-separated list of EnvFitler directives. - pub rust_log: Option, - /// Some("") enables global debug logging. - /// Some("module") enables debug logging for "module". - pub verbose_module: Option, - /// Verbosity level of collected traces. - pub opentelemetry_level: Option, -} - -impl Watcher for LogConfig { - fn reload(instance: Option) -> Result<(), WatchConfigError> { - if let Some(LogConfig { rust_log, verbose_module, opentelemetry_level }) = instance { - Ok(reload(rust_log.as_deref(), verbose_module.as_deref(), opentelemetry_level) - .map_err(|e| into_config_err(e))?) - } else { - Ok(reload(None, None, None).map_err(|e| into_config_err(e))?) - } - } -} - -fn into_config_err(reload_errs: Vec) -> WatchConfigError { - let error_msgs: Vec = reload_errs.iter().map(|e| e.to_string()).collect(); - WatchConfigError::Reload(error_msgs.join("")) -} diff --git a/neard/src/watchers/mod.rs b/neard/src/watchers/mod.rs deleted file mode 100644 index 2b3aba6805c..00000000000 --- a/neard/src/watchers/mod.rs +++ /dev/null @@ -1,62 +0,0 @@ -pub mod dyn_config_watcher; -pub mod log_config_watcher; - -use serde::{Deserialize, Serialize}; -use std::fmt::Debug; -use std::io; -use std::io::ErrorKind; -use std::path::PathBuf; -use tracing::{error, info}; - -pub(crate) enum UpdateBehavior { - UpdateOrReset, - UpdateOnlyIfExists, -} - -#[derive(thiserror::Error, Debug)] -#[non_exhaustive] -pub(crate) enum WatchConfigError { - #[error("Failed to reload the watcher config")] - Reload(String), - #[error("Failed to reload the logging config")] - Parse(#[source] serde_json::Error), - #[error("Can't open or read the logging config file")] - OpenAndRead(#[source] io::Error), -} - -/// Watcher helps to `reload` the change of config -/// main thread will use `update` method to trigger config watchers to reload the config they watch -pub(crate) trait Watcher -where - Self: Debug + for<'a> Deserialize<'a> + Serialize, -{ - fn reload(instance: Option) -> Result<(), WatchConfigError>; - - fn do_update(path: &PathBuf, update_behavior: &UpdateBehavior) -> Result<(), WatchConfigError> { - match std::fs::read_to_string(path) { - Ok(config_str) => match serde_json::from_str::(&config_str) { - Ok(config) => { - info!(target: "neard", config=?config, "Changing the config {path:?}."); - return Self::reload(Some(config)); - } - Err(e) => Err(WatchConfigError::Parse(e)), - }, - Err(err) => match err.kind() { - ErrorKind::NotFound => { - if let UpdateBehavior::UpdateOrReset = update_behavior { - info!(target: "neard", ?err, "Reset the config {path:?} because the logging config file doesn't exist."); - return Self::reload(None); - } - Ok(()) - } - _ => Err(err).map_err(WatchConfigError::OpenAndRead), - }, - } - } - - fn update(path: PathBuf, update_behavior: &UpdateBehavior) { - if let Err(err) = Self::do_update(&path, update_behavior) { - error!(target: "neard", "Failed to update {path:?}: {err:?}."); - } - } -} diff --git a/tools/mock-node/src/setup.rs b/tools/mock-node/src/setup.rs index f95fbd1a3a3..490561ca1e4 100644 --- a/tools/mock-node/src/setup.rs +++ b/tools/mock-node/src/setup.rs @@ -246,6 +246,7 @@ pub fn setup_mock_node( telemetry, None, adv.clone(), + None, ); let view_client = start_view_client(