From df6c683353acecbca51330826dab51f8b788e3b6 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Sun, 12 Mar 2023 17:46:53 +0100 Subject: [PATCH 01/16] Check spawned worker version vs node version before PVF preparation --- Cargo.lock | 1 + cli/src/cli.rs | 2 ++ cli/src/command.rs | 5 +++- node/core/pvf/Cargo.toml | 3 +++ node/core/pvf/build.rs | 19 +++++++++++++++ node/core/pvf/src/error.rs | 6 ++++- node/core/pvf/src/prepare/worker.rs | 36 ++++++++++++++++++++++++++--- node/core/pvf/src/testing.rs | 2 +- node/malus/src/malus.rs | 2 +- 9 files changed, 69 insertions(+), 7 deletions(-) create mode 100644 node/core/pvf/build.rs diff --git a/Cargo.lock b/Cargo.lock index 483104fa08b8..173fd3618b5b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7046,6 +7046,7 @@ dependencies = [ "sp-maybe-compressed-blob", "sp-tracing", "sp-wasm-interface", + "substrate-build-script-utils", "tempfile", "test-parachain-adder", "test-parachain-halt", diff --git a/cli/src/cli.rs b/cli/src/cli.rs index 37083eb91278..8498aa6c38e7 100644 --- a/cli/src/cli.rs +++ b/cli/src/cli.rs @@ -80,6 +80,8 @@ pub enum Subcommand { pub struct ValidationWorkerCommand { /// The path to the validation host's socket. pub socket_path: String, + /// Calling node implementation version + pub node_impl_version: Option, } #[allow(missing_docs)] diff --git a/cli/src/command.rs b/cli/src/command.rs index 0d1a3f81639a..07fbbd725827 100644 --- a/cli/src/command.rs +++ b/cli/src/command.rs @@ -494,7 +494,10 @@ pub fn run() -> Result<()> { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::prepare_worker_entrypoint(&cmd.socket_path); + polkadot_node_core_pvf::prepare_worker_entrypoint( + &cmd.socket_path, + cmd.node_impl_version.as_deref(), + ); Ok(()) } }, diff --git a/node/core/pvf/Cargo.toml b/node/core/pvf/Cargo.toml index a810c3887c96..3f88876283b7 100644 --- a/node/core/pvf/Cargo.toml +++ b/node/core/pvf/Cargo.toml @@ -41,6 +41,9 @@ sp-wasm-interface = { git = "https://github.com/paritytech/substrate", branch = sp-maybe-compressed-blob = { git = "https://github.com/paritytech/substrate", branch = "master" } sp-tracing = { git = "https://github.com/paritytech/substrate", branch = "master" } +[build-dependencies] +substrate-build-script-utils = { git = "https://github.com/paritytech/substrate", branch = "master" } + [target.'cfg(target_os = "linux")'.dependencies] libc = "0.2.139" tikv-jemalloc-ctl = "0.5.0" diff --git a/node/core/pvf/build.rs b/node/core/pvf/build.rs new file mode 100644 index 000000000000..805fa3446f6b --- /dev/null +++ b/node/core/pvf/build.rs @@ -0,0 +1,19 @@ +// Copyright 2017-2023 Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +fn main() { + substrate_build_script_utils::generate_cargo_keys(); +} diff --git a/node/core/pvf/src/error.rs b/node/core/pvf/src/error.rs index 3f642cd6ed24..32e84fb9ca23 100644 --- a/node/core/pvf/src/error.rs +++ b/node/core/pvf/src/error.rs @@ -42,6 +42,8 @@ pub enum PrepareError { /// The response from the worker is received, but the file cannot be renamed (moved) to the final destination /// location. This state is reported by the validation host (not by the worker). RenameTmpFileErr(String), + /// Node and worker version mismatch. May happen if the node binary has been upgraded in-place. + VersionMismatch, } impl PrepareError { @@ -55,7 +57,8 @@ impl PrepareError { use PrepareError::*; match self { Prevalidation(_) | Preparation(_) | Panic(_) => true, - TimedOut | IoErr(_) | CreateTmpFileErr(_) | RenameTmpFileErr(_) => false, + TimedOut | IoErr(_) | CreateTmpFileErr(_) | RenameTmpFileErr(_) | VersionMismatch => + false, } } } @@ -71,6 +74,7 @@ impl fmt::Display for PrepareError { IoErr(err) => write!(f, "prepare: io error while receiving response: {}", err), CreateTmpFileErr(err) => write!(f, "prepare: error creating tmp file: {}", err), RenameTmpFileErr(err) => write!(f, "prepare: error renaming tmp file: {}", err), + VersionMismatch => write!(f, "prepare: node and worker version mismatch"), } } } diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index 962ad2742bf8..74fc15a3ca1e 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -52,7 +52,13 @@ pub async fn spawn( program_path: &Path, spawn_timeout: Duration, ) -> Result<(IdleWorker, WorkerHandle), SpawnErr> { - spawn_with_program_path("prepare", program_path, &["prepare-worker"], spawn_timeout).await + spawn_with_program_path( + "prepare", + program_path, + &["prepare-worker", concat!("--node-impl-version=", env!("SUBSTRATE_CLI_IMPL_VERSION"))], + spawn_timeout, + ) + .await } pub enum Outcome { @@ -175,6 +181,15 @@ async fn handle_response( Ok(result) => result, // Timed out on the child. This should already be logged by the child. Err(PrepareError::TimedOut) => return Outcome::TimedOut, + // Worker reported version mismatch + Err(PrepareError::VersionMismatch) => { + gum::error!( + target: LOG_TARGET, + %worker_pid, + "node and worker version mismatch", + ); + std::process::exit(1); + }, Err(_) => return Outcome::Concluded { worker, result }, }; @@ -342,11 +357,26 @@ async fn recv_response(stream: &mut UnixStream, pid: u32) -> io::Result) { worker_event_loop("prepare", socket_path, |rt_handle, mut stream| async move { + let worker_pid = std::process::id(); + let mut version_checked = false; loop { - let worker_pid = std::process::id(); let (pvf, dest) = recv_request(&mut stream).await?; + if !version_checked { + version_checked = true; + if let Some(version) = node_version { + if version != env!("SUBSTRATE_CLI_IMPL_VERSION") { + gum::error!( + target: LOG_TARGET, + %worker_pid, + "worker: node and worker version mismatch", + ); + send_response(&mut stream, Err(PrepareError::VersionMismatch)).await?; + return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) + } + } + } gum::debug!( target: LOG_TARGET, %worker_pid, diff --git a/node/core/pvf/src/testing.rs b/node/core/pvf/src/testing.rs index e41b769440df..6368f584b28b 100644 --- a/node/core/pvf/src/testing.rs +++ b/node/core/pvf/src/testing.rs @@ -72,7 +72,7 @@ macro_rules! decl_puppet_worker_main { }, "prepare-worker" => { let socket_path = &args[2]; - $crate::prepare_worker_entrypoint(socket_path); + $crate::prepare_worker_entrypoint(socket_path, None); }, "execute-worker" => { let socket_path = &args[2]; diff --git a/node/malus/src/malus.rs b/node/malus/src/malus.rs index bb466d4ba4de..b23b95d9b2fe 100644 --- a/node/malus/src/malus.rs +++ b/node/malus/src/malus.rs @@ -97,7 +97,7 @@ impl MalusCli { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::prepare_worker_entrypoint(&cmd.socket_path); + polkadot_node_core_pvf::prepare_worker_entrypoint(&cmd.socket_path, None); } }, NemesisVariant::PvfExecuteWorker(cmd) => { From 335495bff9509a776895126ebc6472a080f96feb Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Sun, 12 Mar 2023 19:06:22 +0100 Subject: [PATCH 02/16] Address discussions --- cli/src/cli.rs | 3 ++- cli/src/command.rs | 2 +- node/core/pvf/src/prepare/worker.rs | 30 ++++++++++++++--------------- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/cli/src/cli.rs b/cli/src/cli.rs index 8498aa6c38e7..d219e0e33f83 100644 --- a/cli/src/cli.rs +++ b/cli/src/cli.rs @@ -81,7 +81,8 @@ pub struct ValidationWorkerCommand { /// The path to the validation host's socket. pub socket_path: String, /// Calling node implementation version - pub node_impl_version: Option, + #[arg(long)] + pub node_impl_version: String, } #[allow(missing_docs)] diff --git a/cli/src/command.rs b/cli/src/command.rs index 07fbbd725827..61744e6758b6 100644 --- a/cli/src/command.rs +++ b/cli/src/command.rs @@ -496,7 +496,7 @@ pub fn run() -> Result<()> { { polkadot_node_core_pvf::prepare_worker_entrypoint( &cmd.socket_path, - cmd.node_impl_version.as_deref(), + Some(&cmd.node_impl_version), ); Ok(()) } diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index 74fc15a3ca1e..c8845e803327 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -55,7 +55,7 @@ pub async fn spawn( spawn_with_program_path( "prepare", program_path, - &["prepare-worker", concat!("--node-impl-version=", env!("SUBSTRATE_CLI_IMPL_VERSION"))], + &["prepare-worker", "--node-impl-version", env!("SUBSTRATE_CLI_IMPL_VERSION")], spawn_timeout, ) .await @@ -360,22 +360,22 @@ async fn recv_response(stream: &mut UnixStream, pid: u32) -> io::Result) { worker_event_loop("prepare", socket_path, |rt_handle, mut stream| async move { let worker_pid = std::process::id(); - let mut version_checked = false; + let version_mismatch = if let Some(version) = node_version { + version != env!("SUBSTRATE_CLI_IMPL_VERSION") + } else { + false + }; + loop { let (pvf, dest) = recv_request(&mut stream).await?; - if !version_checked { - version_checked = true; - if let Some(version) = node_version { - if version != env!("SUBSTRATE_CLI_IMPL_VERSION") { - gum::error!( - target: LOG_TARGET, - %worker_pid, - "worker: node and worker version mismatch", - ); - send_response(&mut stream, Err(PrepareError::VersionMismatch)).await?; - return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) - } - } + if version_mismatch { + gum::error!( + target: LOG_TARGET, + %worker_pid, + "worker: node and worker version mismatch", + ); + send_response(&mut stream, Err(PrepareError::VersionMismatch)).await?; + return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } gum::debug!( target: LOG_TARGET, From b96cc3160ff58db5ff001d8ca0bfea9bd4bdd0f2 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Mon, 13 Mar 2023 23:04:04 +0100 Subject: [PATCH 03/16] Propagate errors and shutdown preparation and execution pipelines properly --- cli/src/cli.rs | 1 + cli/src/command.rs | 5 +- node/core/pvf/src/execute/mod.rs | 2 +- node/core/pvf/src/execute/queue.rs | 82 ++++++++++++++++++------ node/core/pvf/src/execute/worker.rs | 28 ++++++-- node/core/pvf/src/host.rs | 99 +++++++++++++++++++++++++---- node/core/pvf/src/prepare/pool.rs | 24 +++++++ node/core/pvf/src/prepare/queue.rs | 40 +++++++++--- node/core/pvf/src/prepare/worker.rs | 13 ++-- node/core/pvf/src/testing.rs | 2 +- node/core/pvf/src/worker_common.rs | 1 + node/malus/src/malus.rs | 2 +- 12 files changed, 241 insertions(+), 58 deletions(-) diff --git a/cli/src/cli.rs b/cli/src/cli.rs index d219e0e33f83..c78399788a65 100644 --- a/cli/src/cli.rs +++ b/cli/src/cli.rs @@ -79,6 +79,7 @@ pub enum Subcommand { #[derive(Debug, Parser)] pub struct ValidationWorkerCommand { /// The path to the validation host's socket. + #[arg(long)] pub socket_path: String, /// Calling node implementation version #[arg(long)] diff --git a/cli/src/command.rs b/cli/src/command.rs index 61744e6758b6..e6eaf6f09562 100644 --- a/cli/src/command.rs +++ b/cli/src/command.rs @@ -516,7 +516,10 @@ pub fn run() -> Result<()> { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path); + polkadot_node_core_pvf::execute_worker_entrypoint( + &cmd.socket_path, + Some(&cmd.node_impl_version), + ); Ok(()) } }, diff --git a/node/core/pvf/src/execute/mod.rs b/node/core/pvf/src/execute/mod.rs index bc7f035a8b40..20dc273268e7 100644 --- a/node/core/pvf/src/execute/mod.rs +++ b/node/core/pvf/src/execute/mod.rs @@ -23,5 +23,5 @@ mod queue; mod worker; -pub use queue::{start, ToQueue}; +pub use queue::{start, FromQueue, ToQueue}; pub use worker::{worker_entrypoint, Response as ExecuteResponse}; diff --git a/node/core/pvf/src/execute/queue.rs b/node/core/pvf/src/execute/queue.rs index 9a89c46e7a14..b36d3560307c 100644 --- a/node/core/pvf/src/execute/queue.rs +++ b/node/core/pvf/src/execute/queue.rs @@ -57,6 +57,15 @@ pub enum ToQueue { executor_params: ExecutorParams, result_tx: ResultSender, }, + Shutdown, +} + +#[derive(Debug)] +pub enum FromQueue { + // Node and worker version mismatch + VersionMismatch, + // Queue shutdown complete + ShutdownComplete, } struct ExecuteJob { @@ -128,10 +137,13 @@ enum QueueEvent { type Mux = FuturesUnordered>; struct Queue { + running: bool, metrics: Metrics, /// The receiver that receives messages to the pool. to_queue_rx: mpsc::Receiver, + /// The sender for reporting error conditions + from_queue_tx: mpsc::UnboundedSender, program_path: PathBuf, spawn_timeout: Duration, @@ -149,12 +161,15 @@ impl Queue { worker_capacity: usize, spawn_timeout: Duration, to_queue_rx: mpsc::Receiver, + from_queue_tx: mpsc::UnboundedSender, ) -> Self { Self { + running: true, metrics, program_path, spawn_timeout, to_queue_rx, + from_queue_tx, queue: VecDeque::new(), mux: Mux::new(), workers: Workers { @@ -259,23 +274,34 @@ async fn purge_dead(metrics: &Metrics, workers: &mut Workers) { } fn handle_to_queue(queue: &mut Queue, to_queue: ToQueue) { - let ToQueue::Enqueue { artifact, exec_timeout, params, executor_params, result_tx } = to_queue; - gum::debug!( - target: LOG_TARGET, - validation_code_hash = ?artifact.id.code_hash, - "enqueueing an artifact for execution", - ); - queue.metrics.execute_enqueued(); - let job = ExecuteJob { - artifact, - exec_timeout, - params, - executor_params, - result_tx, - waiting_since: Instant::now(), - }; - queue.queue.push_back(job); - queue.try_assign_next_job(None); + match to_queue { + ToQueue::Enqueue { artifact, exec_timeout, params, executor_params, result_tx } => { + if !queue.running { + return + } + gum::debug!( + target: LOG_TARGET, + validation_code_hash = ?artifact.id.code_hash, + "enqueueing an artifact for execution", + ); + queue.metrics.execute_enqueued(); + let job = ExecuteJob { + artifact, + exec_timeout, + params, + executor_params, + result_tx, + waiting_since: Instant::now(), + }; + queue.queue.push_back(job); + queue.try_assign_next_job(None); + }, + ToQueue::Shutdown => { + queue.running = false; + queue.workers.running.clear(); + let _ = queue.from_queue_tx.unbounded_send(FromQueue::ShutdownComplete); + }, + } } async fn handle_mux(queue: &mut Queue, event: QueueEvent) { @@ -295,6 +321,9 @@ fn handle_worker_spawned( handle: WorkerHandle, job: ExecuteJob, ) { + if !queue.running { + return + } queue.metrics.execute_worker().on_spawned(); queue.workers.spawn_inflight -= 1; let worker = queue.workers.running.insert(WorkerData { @@ -337,6 +366,10 @@ fn handle_job_finish( Err(ValidationError::InvalidCandidate(InvalidCandidate::AmbiguousWorkerDeath)), None, ), + Outcome::VersionMismatch => { + let _ = queue.from_queue_tx.unbounded_send(FromQueue::VersionMismatch); + return + }, }; queue.metrics.execute_finished(); @@ -468,8 +501,17 @@ pub fn start( program_path: PathBuf, worker_capacity: usize, spawn_timeout: Duration, -) -> (mpsc::Sender, impl Future) { +) -> (mpsc::Sender, mpsc::UnboundedReceiver, impl Future) { let (to_queue_tx, to_queue_rx) = mpsc::channel(20); - let run = Queue::new(metrics, program_path, worker_capacity, spawn_timeout, to_queue_rx).run(); - (to_queue_tx, run) + let (from_queue_tx, from_queue_rx) = mpsc::unbounded(); + let run = Queue::new( + metrics, + program_path, + worker_capacity, + spawn_timeout, + to_queue_rx, + from_queue_tx, + ) + .run(); + (to_queue_tx, from_queue_rx, run) } diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 5db6a6261cc9..3f34b5b25fd3 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -77,6 +77,9 @@ pub enum Outcome { InternalError { err: String, idle_worker: IdleWorker }, /// The execution time exceeded the hard limit. The worker is terminated. HardTimeout, + /// The worker recognized a version mismatch between node software and worker. + /// Node should be shut down. + VersionMismatch, /// An I/O error happened during communication with the worker. This may mean that the worker /// process already died. The token is not returned in any case. IoErr, @@ -172,6 +175,7 @@ pub async fn start_work( Response::InvalidCandidate(err) => Outcome::InvalidCandidate { err, idle_worker: IdleWorker { stream, pid } }, Response::TimedOut => Outcome::HardTimeout, + Response::VersionMismatch => Outcome::VersionMismatch, Response::InternalError(err) => Outcome::InternalError { err, idle_worker: IdleWorker { stream, pid } }, } @@ -247,6 +251,7 @@ pub enum Response { InvalidCandidate(String), TimedOut, InternalError(String), + VersionMismatch, } impl Response { @@ -261,19 +266,34 @@ impl Response { /// The entrypoint that the spawned execute worker should start with. The `socket_path` specifies /// the path to the socket used to communicate with the host. -pub fn worker_entrypoint(socket_path: &str) { +pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { worker_event_loop("execute", socket_path, |rt_handle, mut stream| async move { - let handshake = recv_handshake(&mut stream).await?; + let worker_pid = std::process::id(); + let version_mismatch = if let Some(version) = node_version { + version != env!("SUBSTRATE_CLI_IMPL_VERSION") + } else { + false + }; + let handshake = recv_handshake(&mut stream).await?; let executor = Arc::new(Executor::new(handshake.executor_params).map_err(|e| { io::Error::new(io::ErrorKind::Other, format!("cannot create executor: {}", e)) })?); loop { let (artifact_path, params, execution_timeout) = recv_request(&mut stream).await?; + if version_mismatch { + gum::error!( + target: LOG_TARGET, + %worker_pid, + "worker: node and worker version mismatch", + ); + send_response(&mut stream, Response::VersionMismatch).await?; + return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) + } gum::debug!( target: LOG_TARGET, - worker_pid = %std::process::id(), + %worker_pid, "worker: validating artifact {}", artifact_path.display(), ); @@ -307,7 +327,7 @@ pub fn worker_entrypoint(socket_path: &str) { // Log if we exceed the timeout and the other thread hasn't finished. gum::warn!( target: LOG_TARGET, - worker_pid = %std::process::id(), + %worker_pid, "execute job took {}ms cpu time, exceeded execute timeout {}ms", cpu_time_elapsed.as_millis(), execution_timeout.as_millis(), diff --git a/node/core/pvf/src/host.rs b/node/core/pvf/src/host.rs index d4071064d41a..0e818c78dc5b 100644 --- a/node/core/pvf/src/host.rs +++ b/node/core/pvf/src/host.rs @@ -207,7 +207,7 @@ pub fn start(config: Config, metrics: Metrics) -> (ValidationHost, impl Future (ValidationHost, impl Future, to_execute_queue_tx: mpsc::Sender, + from_execute_queue_rx: mpsc::UnboundedReceiver, to_sweeper_tx: mpsc::Sender, awaiting_prepare: AwaitingPrepare, @@ -314,6 +316,7 @@ async fn run( mut artifacts, to_host_rx, from_prepare_queue_rx, + from_execute_queue_rx, mut to_prepare_queue_tx, mut to_execute_queue_tx, mut to_sweeper_tx, @@ -341,6 +344,10 @@ async fn run( let mut to_host_rx = to_host_rx.fuse(); let mut from_prepare_queue_rx = from_prepare_queue_rx.fuse(); + let mut from_execute_queue_rx = from_execute_queue_rx.fuse(); + + let mut prep_pipeline_shut_down = false; + let mut exec_pipeline_shut_down = false; loop { // biased to make it behave deterministically for tests. @@ -395,11 +402,23 @@ async fn run( break_if_fatal!(handle_prepare_done( &cache_path, &mut artifacts, + &mut to_prepare_queue_tx, &mut to_execute_queue_tx, &mut awaiting_prepare, from_queue, + &mut prep_pipeline_shut_down, ).await); }, + from_execute_queue = from_execute_queue_rx.next() => { + let from_queue = break_if_fatal!(from_execute_queue.ok_or(Fatal)); + + // Only to report errors that are fatal to the whole node + break_if_fatal!(handle_execute_outcome(&mut to_prepare_queue_tx, &mut to_execute_queue_tx, from_queue, &mut exec_pipeline_shut_down).await); + }, + } + + if prep_pipeline_shut_down && exec_pipeline_shut_down { + std::process::exit(1); } } } @@ -631,12 +650,43 @@ async fn handle_heads_up( async fn handle_prepare_done( cache_path: &Path, artifacts: &mut Artifacts, + prepare_queue: &mut mpsc::Sender, execute_queue: &mut mpsc::Sender, awaiting_prepare: &mut AwaitingPrepare, from_queue: prepare::FromQueue, + shut_down: &mut bool, ) -> Result<(), Fatal> { - let prepare::FromQueue { artifact_id, result } = from_queue; + match from_queue { + prepare::FromQueue::Outcome { artifact_id, result } => + handle_prepare_outcome( + cache_path, + artifacts, + execute_queue, + awaiting_prepare, + artifact_id, + result, + ) + .await, + prepare::FromQueue::VersionMismatch => { + prepare_queue.send(prepare::ToQueue::Shutdown).await.map_err(|_| Fatal)?; + execute_queue.send(execute::ToQueue::Shutdown).await.map_err(|_| Fatal)?; + Ok(()) + }, + prepare::FromQueue::ShutdownComplete => { + *shut_down = true; + Ok(()) + }, + } +} +async fn handle_prepare_outcome( + cache_path: &Path, + artifacts: &mut Artifacts, + execute_queue: &mut mpsc::Sender, + awaiting_prepare: &mut AwaitingPrepare, + artifact_id: ArtifactId, + result: PrepareResult, +) -> Result<(), Fatal> { // Make some sanity checks and extract the current state. let state = match artifacts.artifact_state_mut(&artifact_id) { None => { @@ -731,6 +781,25 @@ async fn handle_prepare_done( Ok(()) } +async fn handle_execute_outcome( + prepare_queue: &mut mpsc::Sender, + execute_queue: &mut mpsc::Sender, + from_queue: execute::FromQueue, + shut_down: &mut bool, +) -> Result<(), Fatal> { + match from_queue { + execute::FromQueue::VersionMismatch => { + prepare_queue.send(prepare::ToQueue::Shutdown).await.map_err(|_| Fatal)?; + execute_queue.send(execute::ToQueue::Shutdown).await.map_err(|_| Fatal)?; + Ok(()) + }, + execute::FromQueue::ShutdownComplete => { + *shut_down = true; + Ok(()) + }, + } +} + async fn send_prepare( prepare_queue: &mut mpsc::Sender, to_queue: prepare::ToQueue, @@ -890,6 +959,7 @@ pub(crate) mod tests { let (to_prepare_queue_tx, to_prepare_queue_rx) = mpsc::channel(10); let (from_prepare_queue_tx, from_prepare_queue_rx) = mpsc::unbounded(); let (to_execute_queue_tx, to_execute_queue_rx) = mpsc::channel(10); + let (_from_execute_queue_tx, from_execute_queue_rx) = mpsc::unbounded(); let (to_sweeper_tx, to_sweeper_rx) = mpsc::channel(10); let run = run(Inner { @@ -901,6 +971,7 @@ pub(crate) mod tests { to_prepare_queue_tx, from_prepare_queue_rx, to_execute_queue_tx, + from_execute_queue_rx, to_sweeper_tx, awaiting_prepare: AwaitingPrepare::default(), }) @@ -1117,7 +1188,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) @@ -1133,7 +1204,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(2), result: Ok(PrepareStats::default()), }) @@ -1185,7 +1256,7 @@ pub(crate) mod tests { ); // Send `Ok` right away and poll the host. test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) @@ -1209,7 +1280,7 @@ pub(crate) mod tests { prepare::ToQueue::Enqueue { .. } ); test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(2), result: Err(PrepareError::TimedOut), }) @@ -1255,7 +1326,7 @@ pub(crate) mod tests { // Suppose the preparation failed, the execution queue is empty and both // "clients" receive their results. test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1292,7 +1363,7 @@ pub(crate) mod tests { prepare::ToQueue::Enqueue { .. } ); test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(2), result: Ok(PrepareStats::default()), }) @@ -1327,7 +1398,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1394,7 +1465,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1446,7 +1517,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) @@ -1496,7 +1567,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Err(PrepareError::Prevalidation("reproducible error".into())), }) @@ -1574,7 +1645,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1622,7 +1693,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue { + .send(prepare::FromQueue::Outcome { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) diff --git a/node/core/pvf/src/prepare/pool.rs b/node/core/pvf/src/prepare/pool.rs index 8587fcb1a6b6..6d7a2bc80879 100644 --- a/node/core/pvf/src/prepare/pool.rs +++ b/node/core/pvf/src/prepare/pool.rs @@ -66,6 +66,9 @@ pub enum ToPool { /// In either case, the worker is considered busy and no further `StartWork` messages should be /// sent until either `Concluded` or `Rip` message is received. StartWork { worker: Worker, pvf: PvfPrepData, artifact_path: PathBuf }, + + /// Shutdown the pool + Shutdown, } /// A message sent from pool to its client. @@ -87,6 +90,12 @@ pub enum FromPool { /// The given worker ceased to exist. Rip(Worker), + + /// Worker and node version mismatch. + VersionMismatch, + + /// Pool has killed all the workers and is ready for node shutdown. + ShutdownComplete, } struct WorkerData { @@ -103,6 +112,7 @@ impl fmt::Debug for WorkerData { enum PoolEvent { Spawn(IdleWorker, WorkerHandle), StartWork(Worker, Outcome), + ShutdownComplete, } type Mux = FuturesUnordered>; @@ -243,6 +253,11 @@ fn handle_to_pool( // It may be absent if it were previously already removed by `purge_dead`. let _ = attempt_retire(metrics, spawned, worker); }, + ToPool::Shutdown => { + gum::debug!(target: LOG_TARGET, "shutting down preparation pool"); + spawned.clear(); + mux.push(Box::pin(futures::future::ready(PoolEvent::ShutdownComplete))); + }, } } @@ -347,10 +362,19 @@ fn handle_mux( )?; } + Ok(()) + }, + Outcome::VersionMismatch => { + reply(from_pool, FromPool::VersionMismatch)?; + Ok(()) }, } }, + PoolEvent::ShutdownComplete => { + reply(from_pool, FromPool::ShutdownComplete)?; + Ok(()) + }, } } diff --git a/node/core/pvf/src/prepare/queue.rs b/node/core/pvf/src/prepare/queue.rs index 38db463a33b7..a9f185b76ef1 100644 --- a/node/core/pvf/src/prepare/queue.rs +++ b/node/core/pvf/src/prepare/queue.rs @@ -38,17 +38,25 @@ pub enum ToQueue { /// Note that it is incorrect to enqueue the same PVF again without first receiving the /// [`FromQueue`] response. Enqueue { priority: Priority, pvf: PvfPrepData }, + /// Instructs to shutdown the queue + Shutdown, } /// A response from queue. #[derive(Debug)] -pub struct FromQueue { +pub enum FromQueue { /// Identifier of an artifact. - pub(crate) artifact_id: ArtifactId, + // pub(crate) artifact_id: ArtifactId, /// Outcome of the PVF processing. [`Ok`] indicates that compiled artifact /// is successfully stored on disk. Otherwise, an [error](crate::error::PrepareError) /// is supplied. - pub(crate) result: PrepareResult, + // pub(crate) result: PrepareResult, + Outcome { + artifact_id: ArtifactId, + result: PrepareResult, + }, + VersionMismatch, + ShutdownComplete, } #[derive(Default)] @@ -132,6 +140,8 @@ impl Unscheduled { } struct Queue { + running: bool, + metrics: Metrics, to_queue_rx: mpsc::Receiver, @@ -172,6 +182,7 @@ impl Queue { from_pool_rx: mpsc::UnboundedReceiver, ) -> Self { Self { + running: true, metrics, to_queue_rx, from_queue_tx, @@ -210,8 +221,13 @@ impl Queue { async fn handle_to_queue(queue: &mut Queue, to_queue: ToQueue) -> Result<(), Fatal> { match to_queue { - ToQueue::Enqueue { priority, pvf } => { - handle_enqueue(queue, priority, pvf).await?; + ToQueue::Enqueue { priority, pvf } => + if queue.running { + handle_enqueue(queue, priority, pvf).await?; + }, + ToQueue::Shutdown => { + queue.running = false; + queue.to_pool_tx.send(pool::ToPool::Shutdown).await.map_err(|_| Fatal)?; }, } Ok(()) @@ -274,6 +290,14 @@ async fn handle_from_pool(queue: &mut Queue, from_pool: pool::FromPool) -> Resul Concluded { worker, rip, result } => handle_worker_concluded(queue, worker, rip, result).await?, Rip(worker) => handle_worker_rip(queue, worker).await?, + VersionMismatch => queue + .from_queue_tx + .unbounded_send(FromQueue::VersionMismatch) + .map_err(|_| Fatal)?, + ShutdownComplete => queue + .from_queue_tx + .unbounded_send(FromQueue::ShutdownComplete) + .map_err(|_| Fatal)?, } Ok(()) } @@ -351,7 +375,7 @@ async fn handle_worker_concluded( "prepare worker concluded", ); - reply(&mut queue.from_queue_tx, FromQueue { artifact_id, result })?; + reply(&mut queue.from_queue_tx, FromQueue::Outcome { artifact_id, result })?; // Figure out what to do with the worker. if rip { @@ -616,7 +640,7 @@ mod tests { result: Ok(PrepareStats::default()), }); - assert_eq!(test.poll_and_recv_from_queue().await.artifact_id, pvf(1).as_artifact_id()); + assert_matches!(test.poll_and_recv_from_queue().await, FromQueue::Outcome { artifact_id, .. } if artifact_id == pvf(1).as_artifact_id()); } #[tokio::test] @@ -735,7 +759,7 @@ mod tests { // Since there is still work, the queue requested one extra worker to spawn to handle the // remaining enqueued work items. assert_eq!(test.poll_and_recv_to_pool().await, pool::ToPool::Spawn); - assert_eq!(test.poll_and_recv_from_queue().await.artifact_id, pvf(1).as_artifact_id()); + assert_matches!(test.poll_and_recv_from_queue().await, FromQueue::Outcome { artifact_id, .. } if artifact_id == pvf(1).as_artifact_id()); } #[tokio::test] diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index c8845e803327..1b17af4a53ce 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -76,6 +76,10 @@ pub enum Outcome { /// /// The worker is no longer usable and should be killed. TimedOut, + /// The worker recognized a version mismatch between node software and worker. + /// + /// Node should be shut down. + VersionMismatch, /// An IO error occurred while receiving the result from the worker process. /// /// This doesn't return an idle worker instance, thus this worker is no longer usable. @@ -182,14 +186,7 @@ async fn handle_response( // Timed out on the child. This should already be logged by the child. Err(PrepareError::TimedOut) => return Outcome::TimedOut, // Worker reported version mismatch - Err(PrepareError::VersionMismatch) => { - gum::error!( - target: LOG_TARGET, - %worker_pid, - "node and worker version mismatch", - ); - std::process::exit(1); - }, + Err(PrepareError::VersionMismatch) => return Outcome::VersionMismatch, Err(_) => return Outcome::Concluded { worker, result }, }; diff --git a/node/core/pvf/src/testing.rs b/node/core/pvf/src/testing.rs index 6368f584b28b..0370349e433d 100644 --- a/node/core/pvf/src/testing.rs +++ b/node/core/pvf/src/testing.rs @@ -76,7 +76,7 @@ macro_rules! decl_puppet_worker_main { }, "execute-worker" => { let socket_path = &args[2]; - $crate::execute_worker_entrypoint(socket_path); + $crate::execute_worker_entrypoint(socket_path, None); }, other => panic!("unknown subcommand: {}", other), } diff --git a/node/core/pvf/src/worker_common.rs b/node/core/pvf/src/worker_common.rs index 430a6950fb4f..662e5de5303f 100644 --- a/node/core/pvf/src/worker_common.rs +++ b/node/core/pvf/src/worker_common.rs @@ -280,6 +280,7 @@ impl WorkerHandle { ) -> io::Result { let mut child = process::Command::new(program.as_ref()) .args(extra_args) + .arg("--socket-path") .arg(socket_path.as_ref().as_os_str()) .stdout(std::process::Stdio::piped()) .kill_on_drop(true) diff --git a/node/malus/src/malus.rs b/node/malus/src/malus.rs index b23b95d9b2fe..2c10f75beb5a 100644 --- a/node/malus/src/malus.rs +++ b/node/malus/src/malus.rs @@ -108,7 +108,7 @@ impl MalusCli { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path); + polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path, None); } }, } From 54b11c670dc100f0a59095a600432a9fb95eec5d Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Tue, 14 Mar 2023 21:27:09 +0100 Subject: [PATCH 04/16] Add logs; Fix execution worker checks --- node/core/pvf/src/execute/worker.rs | 2 +- node/core/pvf/src/host.rs | 30 +++++++++++++++++++++++++++-- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 3f34b5b25fd3..dcf618f15a8d 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -48,7 +48,7 @@ pub async fn spawn( spawn_timeout: Duration, ) -> Result<(IdleWorker, WorkerHandle), SpawnErr> { let (mut idle_worker, worker_handle) = - spawn_with_program_path("execute", program_path, &["execute-worker"], spawn_timeout) + spawn_with_program_path("execute", program_path, &["execute-worker", "--node-impl-version", env!("SUBSTRATE_CLI_IMPL_VERSION")], spawn_timeout) .await?; send_handshake(&mut idle_worker.stream, Handshake { executor_params }) .await diff --git a/node/core/pvf/src/host.rs b/node/core/pvf/src/host.rs index 0e818c78dc5b..4f1dbc496d2d 100644 --- a/node/core/pvf/src/host.rs +++ b/node/core/pvf/src/host.rs @@ -413,12 +413,22 @@ async fn run( let from_queue = break_if_fatal!(from_execute_queue.ok_or(Fatal)); // Only to report errors that are fatal to the whole node - break_if_fatal!(handle_execute_outcome(&mut to_prepare_queue_tx, &mut to_execute_queue_tx, from_queue, &mut exec_pipeline_shut_down).await); + break_if_fatal!(handle_execute_outcome( + &mut to_prepare_queue_tx, + &mut to_execute_queue_tx, + from_queue, + &mut exec_pipeline_shut_down + ).await); }, } if prep_pipeline_shut_down && exec_pipeline_shut_down { - std::process::exit(1); + gum::error!( + target: LOG_TARGET, + "PVF pipelines shut down, PVF host exiting", + ); + break; + // std::process::exit(1); } } } @@ -668,11 +678,19 @@ async fn handle_prepare_done( ) .await, prepare::FromQueue::VersionMismatch => { + gum::debug!( + target: LOG_TARGET, + "host: prep pipeline reported version mismatch, signaling shutdown" + ); prepare_queue.send(prepare::ToQueue::Shutdown).await.map_err(|_| Fatal)?; execute_queue.send(execute::ToQueue::Shutdown).await.map_err(|_| Fatal)?; Ok(()) }, prepare::FromQueue::ShutdownComplete => { + gum::debug!( + target: LOG_TARGET, + "host: prep pipline signaled shutdown complete" + ); *shut_down = true; Ok(()) }, @@ -789,11 +807,19 @@ async fn handle_execute_outcome( ) -> Result<(), Fatal> { match from_queue { execute::FromQueue::VersionMismatch => { + gum::debug!( + target: LOG_TARGET, + "host: exec pipeline reported version mismatch, signaling shutdown" + ); prepare_queue.send(prepare::ToQueue::Shutdown).await.map_err(|_| Fatal)?; execute_queue.send(execute::ToQueue::Shutdown).await.map_err(|_| Fatal)?; Ok(()) }, execute::FromQueue::ShutdownComplete => { + gum::debug!( + target: LOG_TARGET, + "host: exec pipline signaled shutdown complete" + ); *shut_down = true; Ok(()) }, From 3cd790ff3e2c86e8acc203c6de87f63b93716d35 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Tue, 14 Mar 2023 21:27:29 +0100 Subject: [PATCH 05/16] Revert "Propagate errors and shutdown preparation and execution pipelines properly" This reverts commit b96cc3160ff58db5ff001d8ca0bfea9bd4bdd0f2. --- cli/src/cli.rs | 1 - cli/src/command.rs | 5 +- node/core/pvf/src/execute/mod.rs | 2 +- node/core/pvf/src/execute/queue.rs | 82 ++++++------------------ node/core/pvf/src/execute/worker.rs | 28 ++------ node/core/pvf/src/host.rs | 99 ++++------------------------- node/core/pvf/src/prepare/pool.rs | 24 ------- node/core/pvf/src/prepare/queue.rs | 40 +++--------- node/core/pvf/src/prepare/worker.rs | 13 ++-- node/core/pvf/src/testing.rs | 2 +- node/core/pvf/src/worker_common.rs | 1 - node/malus/src/malus.rs | 2 +- 12 files changed, 58 insertions(+), 241 deletions(-) diff --git a/cli/src/cli.rs b/cli/src/cli.rs index c78399788a65..d219e0e33f83 100644 --- a/cli/src/cli.rs +++ b/cli/src/cli.rs @@ -79,7 +79,6 @@ pub enum Subcommand { #[derive(Debug, Parser)] pub struct ValidationWorkerCommand { /// The path to the validation host's socket. - #[arg(long)] pub socket_path: String, /// Calling node implementation version #[arg(long)] diff --git a/cli/src/command.rs b/cli/src/command.rs index e6eaf6f09562..61744e6758b6 100644 --- a/cli/src/command.rs +++ b/cli/src/command.rs @@ -516,10 +516,7 @@ pub fn run() -> Result<()> { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::execute_worker_entrypoint( - &cmd.socket_path, - Some(&cmd.node_impl_version), - ); + polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path); Ok(()) } }, diff --git a/node/core/pvf/src/execute/mod.rs b/node/core/pvf/src/execute/mod.rs index 20dc273268e7..bc7f035a8b40 100644 --- a/node/core/pvf/src/execute/mod.rs +++ b/node/core/pvf/src/execute/mod.rs @@ -23,5 +23,5 @@ mod queue; mod worker; -pub use queue::{start, FromQueue, ToQueue}; +pub use queue::{start, ToQueue}; pub use worker::{worker_entrypoint, Response as ExecuteResponse}; diff --git a/node/core/pvf/src/execute/queue.rs b/node/core/pvf/src/execute/queue.rs index b36d3560307c..9a89c46e7a14 100644 --- a/node/core/pvf/src/execute/queue.rs +++ b/node/core/pvf/src/execute/queue.rs @@ -57,15 +57,6 @@ pub enum ToQueue { executor_params: ExecutorParams, result_tx: ResultSender, }, - Shutdown, -} - -#[derive(Debug)] -pub enum FromQueue { - // Node and worker version mismatch - VersionMismatch, - // Queue shutdown complete - ShutdownComplete, } struct ExecuteJob { @@ -137,13 +128,10 @@ enum QueueEvent { type Mux = FuturesUnordered>; struct Queue { - running: bool, metrics: Metrics, /// The receiver that receives messages to the pool. to_queue_rx: mpsc::Receiver, - /// The sender for reporting error conditions - from_queue_tx: mpsc::UnboundedSender, program_path: PathBuf, spawn_timeout: Duration, @@ -161,15 +149,12 @@ impl Queue { worker_capacity: usize, spawn_timeout: Duration, to_queue_rx: mpsc::Receiver, - from_queue_tx: mpsc::UnboundedSender, ) -> Self { Self { - running: true, metrics, program_path, spawn_timeout, to_queue_rx, - from_queue_tx, queue: VecDeque::new(), mux: Mux::new(), workers: Workers { @@ -274,34 +259,23 @@ async fn purge_dead(metrics: &Metrics, workers: &mut Workers) { } fn handle_to_queue(queue: &mut Queue, to_queue: ToQueue) { - match to_queue { - ToQueue::Enqueue { artifact, exec_timeout, params, executor_params, result_tx } => { - if !queue.running { - return - } - gum::debug!( - target: LOG_TARGET, - validation_code_hash = ?artifact.id.code_hash, - "enqueueing an artifact for execution", - ); - queue.metrics.execute_enqueued(); - let job = ExecuteJob { - artifact, - exec_timeout, - params, - executor_params, - result_tx, - waiting_since: Instant::now(), - }; - queue.queue.push_back(job); - queue.try_assign_next_job(None); - }, - ToQueue::Shutdown => { - queue.running = false; - queue.workers.running.clear(); - let _ = queue.from_queue_tx.unbounded_send(FromQueue::ShutdownComplete); - }, - } + let ToQueue::Enqueue { artifact, exec_timeout, params, executor_params, result_tx } = to_queue; + gum::debug!( + target: LOG_TARGET, + validation_code_hash = ?artifact.id.code_hash, + "enqueueing an artifact for execution", + ); + queue.metrics.execute_enqueued(); + let job = ExecuteJob { + artifact, + exec_timeout, + params, + executor_params, + result_tx, + waiting_since: Instant::now(), + }; + queue.queue.push_back(job); + queue.try_assign_next_job(None); } async fn handle_mux(queue: &mut Queue, event: QueueEvent) { @@ -321,9 +295,6 @@ fn handle_worker_spawned( handle: WorkerHandle, job: ExecuteJob, ) { - if !queue.running { - return - } queue.metrics.execute_worker().on_spawned(); queue.workers.spawn_inflight -= 1; let worker = queue.workers.running.insert(WorkerData { @@ -366,10 +337,6 @@ fn handle_job_finish( Err(ValidationError::InvalidCandidate(InvalidCandidate::AmbiguousWorkerDeath)), None, ), - Outcome::VersionMismatch => { - let _ = queue.from_queue_tx.unbounded_send(FromQueue::VersionMismatch); - return - }, }; queue.metrics.execute_finished(); @@ -501,17 +468,8 @@ pub fn start( program_path: PathBuf, worker_capacity: usize, spawn_timeout: Duration, -) -> (mpsc::Sender, mpsc::UnboundedReceiver, impl Future) { +) -> (mpsc::Sender, impl Future) { let (to_queue_tx, to_queue_rx) = mpsc::channel(20); - let (from_queue_tx, from_queue_rx) = mpsc::unbounded(); - let run = Queue::new( - metrics, - program_path, - worker_capacity, - spawn_timeout, - to_queue_rx, - from_queue_tx, - ) - .run(); - (to_queue_tx, from_queue_rx, run) + let run = Queue::new(metrics, program_path, worker_capacity, spawn_timeout, to_queue_rx).run(); + (to_queue_tx, run) } diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 3f34b5b25fd3..5db6a6261cc9 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -77,9 +77,6 @@ pub enum Outcome { InternalError { err: String, idle_worker: IdleWorker }, /// The execution time exceeded the hard limit. The worker is terminated. HardTimeout, - /// The worker recognized a version mismatch between node software and worker. - /// Node should be shut down. - VersionMismatch, /// An I/O error happened during communication with the worker. This may mean that the worker /// process already died. The token is not returned in any case. IoErr, @@ -175,7 +172,6 @@ pub async fn start_work( Response::InvalidCandidate(err) => Outcome::InvalidCandidate { err, idle_worker: IdleWorker { stream, pid } }, Response::TimedOut => Outcome::HardTimeout, - Response::VersionMismatch => Outcome::VersionMismatch, Response::InternalError(err) => Outcome::InternalError { err, idle_worker: IdleWorker { stream, pid } }, } @@ -251,7 +247,6 @@ pub enum Response { InvalidCandidate(String), TimedOut, InternalError(String), - VersionMismatch, } impl Response { @@ -266,34 +261,19 @@ impl Response { /// The entrypoint that the spawned execute worker should start with. The `socket_path` specifies /// the path to the socket used to communicate with the host. -pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { +pub fn worker_entrypoint(socket_path: &str) { worker_event_loop("execute", socket_path, |rt_handle, mut stream| async move { - let worker_pid = std::process::id(); - let version_mismatch = if let Some(version) = node_version { - version != env!("SUBSTRATE_CLI_IMPL_VERSION") - } else { - false - }; - let handshake = recv_handshake(&mut stream).await?; + let executor = Arc::new(Executor::new(handshake.executor_params).map_err(|e| { io::Error::new(io::ErrorKind::Other, format!("cannot create executor: {}", e)) })?); loop { let (artifact_path, params, execution_timeout) = recv_request(&mut stream).await?; - if version_mismatch { - gum::error!( - target: LOG_TARGET, - %worker_pid, - "worker: node and worker version mismatch", - ); - send_response(&mut stream, Response::VersionMismatch).await?; - return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) - } gum::debug!( target: LOG_TARGET, - %worker_pid, + worker_pid = %std::process::id(), "worker: validating artifact {}", artifact_path.display(), ); @@ -327,7 +307,7 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { // Log if we exceed the timeout and the other thread hasn't finished. gum::warn!( target: LOG_TARGET, - %worker_pid, + worker_pid = %std::process::id(), "execute job took {}ms cpu time, exceeded execute timeout {}ms", cpu_time_elapsed.as_millis(), execution_timeout.as_millis(), diff --git a/node/core/pvf/src/host.rs b/node/core/pvf/src/host.rs index 0e818c78dc5b..d4071064d41a 100644 --- a/node/core/pvf/src/host.rs +++ b/node/core/pvf/src/host.rs @@ -207,7 +207,7 @@ pub fn start(config: Config, metrics: Metrics) -> (ValidationHost, impl Future (ValidationHost, impl Future, to_execute_queue_tx: mpsc::Sender, - from_execute_queue_rx: mpsc::UnboundedReceiver, to_sweeper_tx: mpsc::Sender, awaiting_prepare: AwaitingPrepare, @@ -316,7 +314,6 @@ async fn run( mut artifacts, to_host_rx, from_prepare_queue_rx, - from_execute_queue_rx, mut to_prepare_queue_tx, mut to_execute_queue_tx, mut to_sweeper_tx, @@ -344,10 +341,6 @@ async fn run( let mut to_host_rx = to_host_rx.fuse(); let mut from_prepare_queue_rx = from_prepare_queue_rx.fuse(); - let mut from_execute_queue_rx = from_execute_queue_rx.fuse(); - - let mut prep_pipeline_shut_down = false; - let mut exec_pipeline_shut_down = false; loop { // biased to make it behave deterministically for tests. @@ -402,23 +395,11 @@ async fn run( break_if_fatal!(handle_prepare_done( &cache_path, &mut artifacts, - &mut to_prepare_queue_tx, &mut to_execute_queue_tx, &mut awaiting_prepare, from_queue, - &mut prep_pipeline_shut_down, ).await); }, - from_execute_queue = from_execute_queue_rx.next() => { - let from_queue = break_if_fatal!(from_execute_queue.ok_or(Fatal)); - - // Only to report errors that are fatal to the whole node - break_if_fatal!(handle_execute_outcome(&mut to_prepare_queue_tx, &mut to_execute_queue_tx, from_queue, &mut exec_pipeline_shut_down).await); - }, - } - - if prep_pipeline_shut_down && exec_pipeline_shut_down { - std::process::exit(1); } } } @@ -650,43 +631,12 @@ async fn handle_heads_up( async fn handle_prepare_done( cache_path: &Path, artifacts: &mut Artifacts, - prepare_queue: &mut mpsc::Sender, execute_queue: &mut mpsc::Sender, awaiting_prepare: &mut AwaitingPrepare, from_queue: prepare::FromQueue, - shut_down: &mut bool, ) -> Result<(), Fatal> { - match from_queue { - prepare::FromQueue::Outcome { artifact_id, result } => - handle_prepare_outcome( - cache_path, - artifacts, - execute_queue, - awaiting_prepare, - artifact_id, - result, - ) - .await, - prepare::FromQueue::VersionMismatch => { - prepare_queue.send(prepare::ToQueue::Shutdown).await.map_err(|_| Fatal)?; - execute_queue.send(execute::ToQueue::Shutdown).await.map_err(|_| Fatal)?; - Ok(()) - }, - prepare::FromQueue::ShutdownComplete => { - *shut_down = true; - Ok(()) - }, - } -} + let prepare::FromQueue { artifact_id, result } = from_queue; -async fn handle_prepare_outcome( - cache_path: &Path, - artifacts: &mut Artifacts, - execute_queue: &mut mpsc::Sender, - awaiting_prepare: &mut AwaitingPrepare, - artifact_id: ArtifactId, - result: PrepareResult, -) -> Result<(), Fatal> { // Make some sanity checks and extract the current state. let state = match artifacts.artifact_state_mut(&artifact_id) { None => { @@ -781,25 +731,6 @@ async fn handle_prepare_outcome( Ok(()) } -async fn handle_execute_outcome( - prepare_queue: &mut mpsc::Sender, - execute_queue: &mut mpsc::Sender, - from_queue: execute::FromQueue, - shut_down: &mut bool, -) -> Result<(), Fatal> { - match from_queue { - execute::FromQueue::VersionMismatch => { - prepare_queue.send(prepare::ToQueue::Shutdown).await.map_err(|_| Fatal)?; - execute_queue.send(execute::ToQueue::Shutdown).await.map_err(|_| Fatal)?; - Ok(()) - }, - execute::FromQueue::ShutdownComplete => { - *shut_down = true; - Ok(()) - }, - } -} - async fn send_prepare( prepare_queue: &mut mpsc::Sender, to_queue: prepare::ToQueue, @@ -959,7 +890,6 @@ pub(crate) mod tests { let (to_prepare_queue_tx, to_prepare_queue_rx) = mpsc::channel(10); let (from_prepare_queue_tx, from_prepare_queue_rx) = mpsc::unbounded(); let (to_execute_queue_tx, to_execute_queue_rx) = mpsc::channel(10); - let (_from_execute_queue_tx, from_execute_queue_rx) = mpsc::unbounded(); let (to_sweeper_tx, to_sweeper_rx) = mpsc::channel(10); let run = run(Inner { @@ -971,7 +901,6 @@ pub(crate) mod tests { to_prepare_queue_tx, from_prepare_queue_rx, to_execute_queue_tx, - from_execute_queue_rx, to_sweeper_tx, awaiting_prepare: AwaitingPrepare::default(), }) @@ -1188,7 +1117,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) @@ -1204,7 +1133,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(2), result: Ok(PrepareStats::default()), }) @@ -1256,7 +1185,7 @@ pub(crate) mod tests { ); // Send `Ok` right away and poll the host. test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) @@ -1280,7 +1209,7 @@ pub(crate) mod tests { prepare::ToQueue::Enqueue { .. } ); test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(2), result: Err(PrepareError::TimedOut), }) @@ -1326,7 +1255,7 @@ pub(crate) mod tests { // Suppose the preparation failed, the execution queue is empty and both // "clients" receive their results. test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1363,7 +1292,7 @@ pub(crate) mod tests { prepare::ToQueue::Enqueue { .. } ); test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(2), result: Ok(PrepareStats::default()), }) @@ -1398,7 +1327,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1465,7 +1394,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1517,7 +1446,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) @@ -1567,7 +1496,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Err(PrepareError::Prevalidation("reproducible error".into())), }) @@ -1645,7 +1574,7 @@ pub(crate) mod tests { ); // Send a PrepareError. test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Err(PrepareError::TimedOut), }) @@ -1693,7 +1622,7 @@ pub(crate) mod tests { ); test.from_prepare_queue_tx - .send(prepare::FromQueue::Outcome { + .send(prepare::FromQueue { artifact_id: artifact_id(1), result: Ok(PrepareStats::default()), }) diff --git a/node/core/pvf/src/prepare/pool.rs b/node/core/pvf/src/prepare/pool.rs index 6d7a2bc80879..8587fcb1a6b6 100644 --- a/node/core/pvf/src/prepare/pool.rs +++ b/node/core/pvf/src/prepare/pool.rs @@ -66,9 +66,6 @@ pub enum ToPool { /// In either case, the worker is considered busy and no further `StartWork` messages should be /// sent until either `Concluded` or `Rip` message is received. StartWork { worker: Worker, pvf: PvfPrepData, artifact_path: PathBuf }, - - /// Shutdown the pool - Shutdown, } /// A message sent from pool to its client. @@ -90,12 +87,6 @@ pub enum FromPool { /// The given worker ceased to exist. Rip(Worker), - - /// Worker and node version mismatch. - VersionMismatch, - - /// Pool has killed all the workers and is ready for node shutdown. - ShutdownComplete, } struct WorkerData { @@ -112,7 +103,6 @@ impl fmt::Debug for WorkerData { enum PoolEvent { Spawn(IdleWorker, WorkerHandle), StartWork(Worker, Outcome), - ShutdownComplete, } type Mux = FuturesUnordered>; @@ -253,11 +243,6 @@ fn handle_to_pool( // It may be absent if it were previously already removed by `purge_dead`. let _ = attempt_retire(metrics, spawned, worker); }, - ToPool::Shutdown => { - gum::debug!(target: LOG_TARGET, "shutting down preparation pool"); - spawned.clear(); - mux.push(Box::pin(futures::future::ready(PoolEvent::ShutdownComplete))); - }, } } @@ -362,19 +347,10 @@ fn handle_mux( )?; } - Ok(()) - }, - Outcome::VersionMismatch => { - reply(from_pool, FromPool::VersionMismatch)?; - Ok(()) }, } }, - PoolEvent::ShutdownComplete => { - reply(from_pool, FromPool::ShutdownComplete)?; - Ok(()) - }, } } diff --git a/node/core/pvf/src/prepare/queue.rs b/node/core/pvf/src/prepare/queue.rs index a9f185b76ef1..38db463a33b7 100644 --- a/node/core/pvf/src/prepare/queue.rs +++ b/node/core/pvf/src/prepare/queue.rs @@ -38,25 +38,17 @@ pub enum ToQueue { /// Note that it is incorrect to enqueue the same PVF again without first receiving the /// [`FromQueue`] response. Enqueue { priority: Priority, pvf: PvfPrepData }, - /// Instructs to shutdown the queue - Shutdown, } /// A response from queue. #[derive(Debug)] -pub enum FromQueue { +pub struct FromQueue { /// Identifier of an artifact. - // pub(crate) artifact_id: ArtifactId, + pub(crate) artifact_id: ArtifactId, /// Outcome of the PVF processing. [`Ok`] indicates that compiled artifact /// is successfully stored on disk. Otherwise, an [error](crate::error::PrepareError) /// is supplied. - // pub(crate) result: PrepareResult, - Outcome { - artifact_id: ArtifactId, - result: PrepareResult, - }, - VersionMismatch, - ShutdownComplete, + pub(crate) result: PrepareResult, } #[derive(Default)] @@ -140,8 +132,6 @@ impl Unscheduled { } struct Queue { - running: bool, - metrics: Metrics, to_queue_rx: mpsc::Receiver, @@ -182,7 +172,6 @@ impl Queue { from_pool_rx: mpsc::UnboundedReceiver, ) -> Self { Self { - running: true, metrics, to_queue_rx, from_queue_tx, @@ -221,13 +210,8 @@ impl Queue { async fn handle_to_queue(queue: &mut Queue, to_queue: ToQueue) -> Result<(), Fatal> { match to_queue { - ToQueue::Enqueue { priority, pvf } => - if queue.running { - handle_enqueue(queue, priority, pvf).await?; - }, - ToQueue::Shutdown => { - queue.running = false; - queue.to_pool_tx.send(pool::ToPool::Shutdown).await.map_err(|_| Fatal)?; + ToQueue::Enqueue { priority, pvf } => { + handle_enqueue(queue, priority, pvf).await?; }, } Ok(()) @@ -290,14 +274,6 @@ async fn handle_from_pool(queue: &mut Queue, from_pool: pool::FromPool) -> Resul Concluded { worker, rip, result } => handle_worker_concluded(queue, worker, rip, result).await?, Rip(worker) => handle_worker_rip(queue, worker).await?, - VersionMismatch => queue - .from_queue_tx - .unbounded_send(FromQueue::VersionMismatch) - .map_err(|_| Fatal)?, - ShutdownComplete => queue - .from_queue_tx - .unbounded_send(FromQueue::ShutdownComplete) - .map_err(|_| Fatal)?, } Ok(()) } @@ -375,7 +351,7 @@ async fn handle_worker_concluded( "prepare worker concluded", ); - reply(&mut queue.from_queue_tx, FromQueue::Outcome { artifact_id, result })?; + reply(&mut queue.from_queue_tx, FromQueue { artifact_id, result })?; // Figure out what to do with the worker. if rip { @@ -640,7 +616,7 @@ mod tests { result: Ok(PrepareStats::default()), }); - assert_matches!(test.poll_and_recv_from_queue().await, FromQueue::Outcome { artifact_id, .. } if artifact_id == pvf(1).as_artifact_id()); + assert_eq!(test.poll_and_recv_from_queue().await.artifact_id, pvf(1).as_artifact_id()); } #[tokio::test] @@ -759,7 +735,7 @@ mod tests { // Since there is still work, the queue requested one extra worker to spawn to handle the // remaining enqueued work items. assert_eq!(test.poll_and_recv_to_pool().await, pool::ToPool::Spawn); - assert_matches!(test.poll_and_recv_from_queue().await, FromQueue::Outcome { artifact_id, .. } if artifact_id == pvf(1).as_artifact_id()); + assert_eq!(test.poll_and_recv_from_queue().await.artifact_id, pvf(1).as_artifact_id()); } #[tokio::test] diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index 1b17af4a53ce..c8845e803327 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -76,10 +76,6 @@ pub enum Outcome { /// /// The worker is no longer usable and should be killed. TimedOut, - /// The worker recognized a version mismatch between node software and worker. - /// - /// Node should be shut down. - VersionMismatch, /// An IO error occurred while receiving the result from the worker process. /// /// This doesn't return an idle worker instance, thus this worker is no longer usable. @@ -186,7 +182,14 @@ async fn handle_response( // Timed out on the child. This should already be logged by the child. Err(PrepareError::TimedOut) => return Outcome::TimedOut, // Worker reported version mismatch - Err(PrepareError::VersionMismatch) => return Outcome::VersionMismatch, + Err(PrepareError::VersionMismatch) => { + gum::error!( + target: LOG_TARGET, + %worker_pid, + "node and worker version mismatch", + ); + std::process::exit(1); + }, Err(_) => return Outcome::Concluded { worker, result }, }; diff --git a/node/core/pvf/src/testing.rs b/node/core/pvf/src/testing.rs index 0370349e433d..6368f584b28b 100644 --- a/node/core/pvf/src/testing.rs +++ b/node/core/pvf/src/testing.rs @@ -76,7 +76,7 @@ macro_rules! decl_puppet_worker_main { }, "execute-worker" => { let socket_path = &args[2]; - $crate::execute_worker_entrypoint(socket_path, None); + $crate::execute_worker_entrypoint(socket_path); }, other => panic!("unknown subcommand: {}", other), } diff --git a/node/core/pvf/src/worker_common.rs b/node/core/pvf/src/worker_common.rs index 662e5de5303f..430a6950fb4f 100644 --- a/node/core/pvf/src/worker_common.rs +++ b/node/core/pvf/src/worker_common.rs @@ -280,7 +280,6 @@ impl WorkerHandle { ) -> io::Result { let mut child = process::Command::new(program.as_ref()) .args(extra_args) - .arg("--socket-path") .arg(socket_path.as_ref().as_os_str()) .stdout(std::process::Stdio::piped()) .kill_on_drop(true) diff --git a/node/malus/src/malus.rs b/node/malus/src/malus.rs index 2c10f75beb5a..b23b95d9b2fe 100644 --- a/node/malus/src/malus.rs +++ b/node/malus/src/malus.rs @@ -108,7 +108,7 @@ impl MalusCli { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path, None); + polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path); } }, } From acc94f7e297e50f20f0d746fc5fd1a5088bc7015 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 15 Mar 2023 11:24:53 +0100 Subject: [PATCH 06/16] Don't try to shut down; report the condition and exit worker --- cli/src/cli.rs | 1 + cli/src/command.rs | 5 ++++- node/core/pvf/src/execute/worker.rs | 29 ++++++++++++++++++++++------- node/core/pvf/src/prepare/worker.rs | 18 +++++++----------- node/core/pvf/src/testing.rs | 2 +- node/core/pvf/src/worker_common.rs | 1 + node/malus/src/malus.rs | 2 +- 7 files changed, 37 insertions(+), 21 deletions(-) diff --git a/cli/src/cli.rs b/cli/src/cli.rs index d219e0e33f83..c78399788a65 100644 --- a/cli/src/cli.rs +++ b/cli/src/cli.rs @@ -79,6 +79,7 @@ pub enum Subcommand { #[derive(Debug, Parser)] pub struct ValidationWorkerCommand { /// The path to the validation host's socket. + #[arg(long)] pub socket_path: String, /// Calling node implementation version #[arg(long)] diff --git a/cli/src/command.rs b/cli/src/command.rs index 61744e6758b6..e6eaf6f09562 100644 --- a/cli/src/command.rs +++ b/cli/src/command.rs @@ -516,7 +516,10 @@ pub fn run() -> Result<()> { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path); + polkadot_node_core_pvf::execute_worker_entrypoint( + &cmd.socket_path, + Some(&cmd.node_impl_version), + ); Ok(()) } }, diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 5db6a6261cc9..81e2228026f4 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -47,9 +47,13 @@ pub async fn spawn( executor_params: ExecutorParams, spawn_timeout: Duration, ) -> Result<(IdleWorker, WorkerHandle), SpawnErr> { - let (mut idle_worker, worker_handle) = - spawn_with_program_path("execute", program_path, &["execute-worker"], spawn_timeout) - .await?; + let (mut idle_worker, worker_handle) = spawn_with_program_path( + "execute", + program_path, + &["execute-worker", "--node-impl-version", env!("SUBSTRATE_CLI_IMPL_VERSION")], + spawn_timeout, + ) + .await?; send_handshake(&mut idle_worker.stream, Handshake { executor_params }) .await .map_err(|error| { @@ -261,10 +265,21 @@ impl Response { /// The entrypoint that the spawned execute worker should start with. The `socket_path` specifies /// the path to the socket used to communicate with the host. -pub fn worker_entrypoint(socket_path: &str) { +pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { worker_event_loop("execute", socket_path, |rt_handle, mut stream| async move { - let handshake = recv_handshake(&mut stream).await?; + let worker_pid = std::process::id(); + if let Some(version) = node_version { + if version != env!("SUBSTRATE_CLI_IMPL_VERSION") { + gum::error!( + target: LOG_TARGET, + %worker_pid, + "Node and worker version mismatch, node needs restarting", + ); + return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) + } + } + let handshake = recv_handshake(&mut stream).await?; let executor = Arc::new(Executor::new(handshake.executor_params).map_err(|e| { io::Error::new(io::ErrorKind::Other, format!("cannot create executor: {}", e)) })?); @@ -273,7 +288,7 @@ pub fn worker_entrypoint(socket_path: &str) { let (artifact_path, params, execution_timeout) = recv_request(&mut stream).await?; gum::debug!( target: LOG_TARGET, - worker_pid = %std::process::id(), + %worker_pid, "worker: validating artifact {}", artifact_path.display(), ); @@ -307,7 +322,7 @@ pub fn worker_entrypoint(socket_path: &str) { // Log if we exceed the timeout and the other thread hasn't finished. gum::warn!( target: LOG_TARGET, - worker_pid = %std::process::id(), + %worker_pid, "execute job took {}ms cpu time, exceeded execute timeout {}ms", cpu_time_elapsed.as_millis(), execution_timeout.as_millis(), diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index c8845e803327..f9f0701af727 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -360,23 +360,19 @@ async fn recv_response(stream: &mut UnixStream, pid: u32) -> io::Result) { worker_event_loop("prepare", socket_path, |rt_handle, mut stream| async move { let worker_pid = std::process::id(); - let version_mismatch = if let Some(version) = node_version { - version != env!("SUBSTRATE_CLI_IMPL_VERSION") - } else { - false - }; - - loop { - let (pvf, dest) = recv_request(&mut stream).await?; - if version_mismatch { + if let Some(version) = node_version { + if version != env!("SUBSTRATE_CLI_IMPL_VERSION") { gum::error!( target: LOG_TARGET, %worker_pid, - "worker: node and worker version mismatch", + "Node and worker version mismatch, node needs restarting", ); - send_response(&mut stream, Err(PrepareError::VersionMismatch)).await?; return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } + } + + loop { + let (pvf, dest) = recv_request(&mut stream).await?; gum::debug!( target: LOG_TARGET, %worker_pid, diff --git a/node/core/pvf/src/testing.rs b/node/core/pvf/src/testing.rs index 6368f584b28b..0370349e433d 100644 --- a/node/core/pvf/src/testing.rs +++ b/node/core/pvf/src/testing.rs @@ -76,7 +76,7 @@ macro_rules! decl_puppet_worker_main { }, "execute-worker" => { let socket_path = &args[2]; - $crate::execute_worker_entrypoint(socket_path); + $crate::execute_worker_entrypoint(socket_path, None); }, other => panic!("unknown subcommand: {}", other), } diff --git a/node/core/pvf/src/worker_common.rs b/node/core/pvf/src/worker_common.rs index 430a6950fb4f..662e5de5303f 100644 --- a/node/core/pvf/src/worker_common.rs +++ b/node/core/pvf/src/worker_common.rs @@ -280,6 +280,7 @@ impl WorkerHandle { ) -> io::Result { let mut child = process::Command::new(program.as_ref()) .args(extra_args) + .arg("--socket-path") .arg(socket_path.as_ref().as_os_str()) .stdout(std::process::Stdio::piped()) .kill_on_drop(true) diff --git a/node/malus/src/malus.rs b/node/malus/src/malus.rs index b23b95d9b2fe..2c10f75beb5a 100644 --- a/node/malus/src/malus.rs +++ b/node/malus/src/malus.rs @@ -108,7 +108,7 @@ impl MalusCli { #[cfg(not(target_os = "android"))] { - polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path); + polkadot_node_core_pvf::execute_worker_entrypoint(&cmd.socket_path, None); } }, } From 89c789755459e7af61ba15675de7ac20fc86e522 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 15 Mar 2023 11:36:54 +0100 Subject: [PATCH 07/16] Get rid of `VersionMismatch` preparation error --- node/core/pvf/src/error.rs | 6 +----- node/core/pvf/src/prepare/worker.rs | 9 --------- 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/node/core/pvf/src/error.rs b/node/core/pvf/src/error.rs index 32e84fb9ca23..3f642cd6ed24 100644 --- a/node/core/pvf/src/error.rs +++ b/node/core/pvf/src/error.rs @@ -42,8 +42,6 @@ pub enum PrepareError { /// The response from the worker is received, but the file cannot be renamed (moved) to the final destination /// location. This state is reported by the validation host (not by the worker). RenameTmpFileErr(String), - /// Node and worker version mismatch. May happen if the node binary has been upgraded in-place. - VersionMismatch, } impl PrepareError { @@ -57,8 +55,7 @@ impl PrepareError { use PrepareError::*; match self { Prevalidation(_) | Preparation(_) | Panic(_) => true, - TimedOut | IoErr(_) | CreateTmpFileErr(_) | RenameTmpFileErr(_) | VersionMismatch => - false, + TimedOut | IoErr(_) | CreateTmpFileErr(_) | RenameTmpFileErr(_) => false, } } } @@ -74,7 +71,6 @@ impl fmt::Display for PrepareError { IoErr(err) => write!(f, "prepare: io error while receiving response: {}", err), CreateTmpFileErr(err) => write!(f, "prepare: error creating tmp file: {}", err), RenameTmpFileErr(err) => write!(f, "prepare: error renaming tmp file: {}", err), - VersionMismatch => write!(f, "prepare: node and worker version mismatch"), } } } diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index f9f0701af727..dd88650f7fb9 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -181,15 +181,6 @@ async fn handle_response( Ok(result) => result, // Timed out on the child. This should already be logged by the child. Err(PrepareError::TimedOut) => return Outcome::TimedOut, - // Worker reported version mismatch - Err(PrepareError::VersionMismatch) => { - gum::error!( - target: LOG_TARGET, - %worker_pid, - "node and worker version mismatch", - ); - std::process::exit(1); - }, Err(_) => return Outcome::Concluded { worker, result }, }; From fb5ccc9b08df3d34bf56f48dc1c2b97f392358ea Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 15 Mar 2023 19:40:30 +0100 Subject: [PATCH 08/16] Merge master --- .gitlab-ci.yml | 18 +- Cargo.lock | 615 +-- Cargo.toml | 2 +- node/core/candidate-validation/src/lib.rs | 15 +- node/core/pvf/Cargo.toml | 2 +- node/core/pvf/src/artifacts.rs | 4 +- node/core/pvf/src/error.rs | 6 - node/core/pvf/src/execute/queue.rs | 2 +- node/core/pvf/src/execute/worker.rs | 2 +- node/core/pvf/src/executor_intf.rs | 2 +- node/core/pvf/src/host.rs | 2 +- node/core/pvf/src/pvf.rs | 2 +- node/core/pvf/src/testing.rs | 2 +- node/core/pvf/tests/it/main.rs | 2 +- node/core/runtime-api/src/cache.rs | 11 +- node/metrics/Cargo.toml | 2 +- node/network/bridge/Cargo.toml | 1 - node/network/bridge/src/network.rs | 15 +- node/network/protocol/Cargo.toml | 1 - node/network/protocol/src/peer_set.rs | 14 +- node/overseer/Cargo.toml | 4 +- node/overseer/src/lib.rs | 5 +- node/overseer/src/tests.rs | 1 + node/service/src/lib.rs | 3 +- node/service/src/overseer.rs | 2 +- node/subsystem-types/Cargo.toml | 2 +- node/subsystem-types/src/messages.rs | 6 +- node/subsystem-types/src/runtime_client.rs | 4 +- node/subsystem-util/Cargo.toml | 2 +- node/subsystem-util/src/lib.rs | 2 +- node/test/performance-test/src/lib.rs | 2 +- node/test/service/Cargo.toml | 1 - node/test/service/src/lib.rs | 6 +- primitives/src/lib.rs | 17 +- primitives/src/runtime_api.rs | 12 +- .../src/{vstaging => v4}/executor_params.rs | 0 primitives/src/{v2 => v4}/metrics.rs | 0 primitives/src/{v2 => v4}/mod.rs | 7 +- primitives/src/{v2 => v4}/signed.rs | 0 primitives/src/vstaging/mod.rs | 3 - .../src/node/utility/pvf-prechecker.md | 6 + .../implementers-guide/src/pvf-prechecking.md | 4 +- runtime/common/src/assigned_slots.rs | 1 - runtime/common/src/auctions.rs | 1 - runtime/common/src/claims.rs | 1 - runtime/common/src/crowdloan/mod.rs | 1 - runtime/common/src/paras_registrar.rs | 1 - runtime/common/src/paras_sudo_wrapper.rs | 1 - runtime/common/src/purchase.rs | 1 - runtime/common/src/slots/mod.rs | 1 - runtime/kusama/src/lib.rs | 42 +- .../src/weights/pallet_nomination_pools.rs | 312 +- runtime/parachains/src/configuration.rs | 11 +- .../parachains/src/configuration/migration.rs | 4 +- runtime/parachains/src/configuration/tests.rs | 49 +- runtime/parachains/src/dmp.rs | 25 +- runtime/parachains/src/dmp/tests.rs | 8 +- runtime/parachains/src/hrmp.rs | 193 +- runtime/parachains/src/hrmp/tests.rs | 2 +- runtime/parachains/src/inclusion/mod.rs | 1 - runtime/parachains/src/initializer.rs | 1 - runtime/parachains/src/initializer/tests.rs | 11 +- runtime/parachains/src/origin.rs | 1 - runtime/parachains/src/paras/benchmarking.rs | 12 +- runtime/parachains/src/paras/mod.rs | 100 +- runtime/parachains/src/paras/tests.rs | 214 +- runtime/parachains/src/paras_inherent/mod.rs | 1 - .../parachains/src/runtime_api_impl/mod.rs | 2 +- .../src/runtime_api_impl/{v2.rs => v4.rs} | 34 +- .../src/runtime_api_impl/vstaging.rs | 25 - runtime/parachains/src/scheduler.rs | 1 - runtime/parachains/src/scheduler/tests.rs | 2 +- runtime/parachains/src/session_info.rs | 4 +- runtime/parachains/src/shared.rs | 1 - runtime/parachains/src/ump.rs | 43 +- runtime/parachains/src/ump/tests.rs | 20 +- runtime/polkadot/src/lib.rs | 56 +- .../src/weights/pallet_nomination_pools.rs | 304 +- runtime/rococo/src/lib.rs | 28 +- runtime/test-runtime/src/lib.rs | 22 +- runtime/westend/src/lib.rs | 45 +- .../src/weights/pallet_nomination_pools.rs | 308 +- scripts/ci/gitlab/pipeline/test.yml | 3 +- utils/staking-miner/Cargo.lock | 4484 ----------------- xcm/pallet-xcm/src/lib.rs | 1 - xcm/pallet-xcm/src/migration.rs | 4 +- xcm/pallet-xcm/src/mock.rs | 1 - xcm/xcm-simulator/example/src/parachain.rs | 1 - xcm/xcm-simulator/fuzzer/src/parachain.rs | 1 - 89 files changed, 1445 insertions(+), 5749 deletions(-) rename primitives/src/{vstaging => v4}/executor_params.rs (100%) rename primitives/src/{v2 => v4}/metrics.rs (100%) rename primitives/src/{v2 => v4}/mod.rs (99%) rename primitives/src/{v2 => v4}/signed.rs (100%) rename runtime/parachains/src/runtime_api_impl/{v2.rs => v4.rs} (91%) delete mode 100644 utils/staking-miner/Cargo.lock diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml index 7ec774723e7b..a6ebde20a866 100644 --- a/.gitlab-ci.yml +++ b/.gitlab-ci.yml @@ -48,6 +48,11 @@ default: - api_failure interruptible: true +.common-before-script: + before_script: + - !reference [.job-switcher, before_script] + - !reference [.timestamp, before_script] + .collect-artifacts: artifacts: name: "${CI_JOB_NAME}_${CI_COMMIT_REF_NAME}" @@ -68,7 +73,6 @@ default: # they will be used if the job fails .pipeline-stopper-vars: before_script: - - !reference [.job-switcher, before_script] - echo "FAILED_JOB_URL=${CI_JOB_URL}" > pipeline-stopper.env - echo "FAILED_JOB_NAME=${CI_JOB_NAME}" >> pipeline-stopper.env - echo "FAILED_JOB_NAME=${CI_JOB_NAME}" >> pipeline-stopper.env @@ -86,20 +90,20 @@ default: .kubernetes-env: image: "${CI_IMAGE}" before_script: - - !reference [.job-switcher, before_script] + - !reference [.common-before-script, before_script] tags: - kubernetes-parity-build .docker-env: image: "${CI_IMAGE}" before_script: - - !reference [.job-switcher, before_script] + - !reference [.common-before-script, before_script] tags: - linux-docker-vm-c2 .compiler-info: before_script: - - !reference [.job-switcher, before_script] + - !reference [.common-before-script, before_script] - rustup show - cargo --version @@ -154,7 +158,7 @@ default: .build-push-image: before_script: - - !reference [.job-switcher, before_script] + - !reference [.common-before-script, before_script] - test -s ./artifacts/VERSION || exit 1 - test -s ./artifacts/EXTRATAG || exit 1 - VERSION="$(cat ./artifacts/VERSION)" @@ -196,6 +200,10 @@ include: - scripts/ci/gitlab/pipeline/publish.yml # zombienet jobs - scripts/ci/gitlab/pipeline/zombienet.yml + # timestamp handler + - project: parity/infrastructure/ci_cd/shared + ref: v0.1 + file: /common/timestamp.yml #### stage: .post diff --git a/Cargo.lock b/Cargo.lock index 173fd3618b5b..a3df979da1ec 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -466,7 +466,7 @@ dependencies = [ [[package]] name = "binary-merkle-tree" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "hash-db", "log", @@ -667,9 +667,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.8.0" +version = "3.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" +checksum = "0d261e256854913907f67ed06efbc3338dfe6179796deefc1ff763fc1aee5535" [[package]] name = "byte-slice-cast" @@ -2077,6 +2077,19 @@ dependencies = [ "syn", ] +[[package]] +name = "expander" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f360349150728553f92e4c997a16af8915f418d3a0f21b440d34c5632f16ed84" +dependencies = [ + "blake2", + "fs-err", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "eyre" version = "0.6.5" @@ -2282,7 +2295,7 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "fork-tree" version = "3.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", ] @@ -2305,7 +2318,7 @@ checksum = "6c2141d6d6c8512188a7891b4b01590a45f6dac67afb4f255c4124dbb86d4eaa" [[package]] name = "frame-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-support-procedural", @@ -2330,7 +2343,7 @@ dependencies = [ [[package]] name = "frame-benchmarking-cli" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "Inflector", "array-bytes", @@ -2377,7 +2390,7 @@ dependencies = [ [[package]] name = "frame-election-provider-solution-type" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro-crate", "proc-macro2", @@ -2388,7 +2401,7 @@ dependencies = [ [[package]] name = "frame-election-provider-support" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-election-provider-solution-type", "frame-support", @@ -2405,7 +2418,7 @@ dependencies = [ [[package]] name = "frame-executive" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -2434,7 +2447,7 @@ dependencies = [ [[package]] name = "frame-remote-externalities" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "log", @@ -2450,7 +2463,7 @@ dependencies = [ [[package]] name = "frame-support" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "bitflags", "environmental", @@ -2483,7 +2496,7 @@ dependencies = [ [[package]] name = "frame-support-procedural" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "Inflector", "cfg-expr", @@ -2498,7 +2511,7 @@ dependencies = [ [[package]] name = "frame-support-procedural-tools" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support-procedural-tools-derive", "proc-macro-crate", @@ -2510,7 +2523,7 @@ dependencies = [ [[package]] name = "frame-support-procedural-tools-derive" version = "3.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro2", "quote", @@ -2520,7 +2533,7 @@ dependencies = [ [[package]] name = "frame-support-test" version = "3.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -2544,7 +2557,7 @@ dependencies = [ [[package]] name = "frame-support-test-pallet" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -2555,7 +2568,7 @@ dependencies = [ [[package]] name = "frame-system" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "log", @@ -2573,7 +2586,7 @@ dependencies = [ [[package]] name = "frame-system-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -2588,7 +2601,7 @@ dependencies = [ [[package]] name = "frame-system-rpc-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "sp-api", @@ -2597,7 +2610,7 @@ dependencies = [ [[package]] name = "frame-try-runtime" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "parity-scale-codec", @@ -2779,7 +2792,7 @@ dependencies = [ [[package]] name = "generate-bags" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "chrono", "frame-election-provider-support", @@ -2880,9 +2893,9 @@ checksum = "dec7af912d60cdbd3677c1af9352ebae6fb8394d165568a2234df0fa00f87793" [[package]] name = "git2" -version = "0.16.0" +version = "0.16.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be36bc9e0546df253c0cc41fd0af34f5e92845ad8509462ec76672fac6997f5b" +checksum = "ccf7f68c2995f392c49fffb4f95ae2c873297830eb25c6bc4c114ce8f4562acc" dependencies = [ "bitflags", "libc", @@ -2962,9 +2975,9 @@ dependencies = [ [[package]] name = "hash-db" -version = "0.15.2" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d23bd4e7b5eda0d0f3a307e8b381fdc8ba9000f26fbe912250c0a4cc3956364a" +checksum = "8e7d7786361d7425ae2fe4f9e407eb0efaa0840f5212d109cc018c40c35c6ab4" [[package]] name = "hash256-std-hasher" @@ -3571,7 +3584,7 @@ checksum = "67c21572b4949434e4fc1e1978b99c5f77064153c59d998bf13ecd96fb5ecba7" [[package]] name = "kusama-runtime" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "frame-benchmarking", @@ -3676,7 +3689,7 @@ dependencies = [ [[package]] name = "kusama-runtime-constants" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "polkadot-primitives", @@ -3749,9 +3762,9 @@ checksum = "201de327520df007757c1f0adce6e827fe8562fbc28bfd9c15571c66ca1f5f79" [[package]] name = "libgit2-sys" -version = "0.14.1+1.5.0" +version = "0.14.2+1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a07fb2692bc3593bda59de45a502bb3071659f2c515e28c71e728306b038e17" +checksum = "7f3d95f6b51075fe9810a7ae22c7095f12b98005ab364d8544797a825ce946a4" dependencies = [ "cc", "libc", @@ -4476,12 +4489,11 @@ dependencies = [ [[package]] name = "memory-db" -version = "0.31.0" +version = "0.32.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e0c7cba9ce19ac7ffd2053ac9f49843bbd3f4318feedfd74e85c19d5fb0ba66" +checksum = "808b50db46293432a45e63bc15ea51e0ab4c0a1647b8eb114e31a3e698dd6fbe" dependencies = [ "hash-db", - "hashbrown 0.12.3", ] [[package]] @@ -4559,7 +4571,7 @@ dependencies = [ [[package]] name = "mmr-gadget" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "log", @@ -4578,7 +4590,7 @@ dependencies = [ [[package]] name = "mmr-rpc" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "anyhow", "jsonrpsee", @@ -5055,9 +5067,9 @@ dependencies = [ [[package]] name = "orchestra" -version = "0.0.4" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17e7d5b6bb115db09390bed8842c94180893dd83df3dfce7354f2a2aa090a4ee" +checksum = "ecae97f74ddeb5c6ff00c98ef368423b4ec9a4d5ab48c0d909e84b7594b09718" dependencies = [ "async-trait", "dyn-clonable", @@ -5072,9 +5084,9 @@ dependencies = [ [[package]] name = "orchestra-proc-macro" -version = "0.0.4" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2af4dabb2286b0be0e9711d2d24e25f6217048b71210cffd3daddc3b5c84e1f" +checksum = "08a9846c70b1df02657dbe5b54fadbad0ea3b944623ebf3fcbf7f33ef859dcba" dependencies = [ "expander 0.0.6", "itertools", @@ -5140,7 +5152,7 @@ dependencies = [ [[package]] name = "pallet-assets" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5155,7 +5167,7 @@ dependencies = [ [[package]] name = "pallet-authority-discovery" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5171,7 +5183,7 @@ dependencies = [ [[package]] name = "pallet-authorship" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5185,7 +5197,7 @@ dependencies = [ [[package]] name = "pallet-babe" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5209,7 +5221,7 @@ dependencies = [ [[package]] name = "pallet-bags-list" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5229,7 +5241,7 @@ dependencies = [ [[package]] name = "pallet-bags-list-remote-tests" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-election-provider-support", "frame-remote-externalities", @@ -5248,7 +5260,7 @@ dependencies = [ [[package]] name = "pallet-balances" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5263,7 +5275,7 @@ dependencies = [ [[package]] name = "pallet-beefy" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5282,7 +5294,7 @@ dependencies = [ [[package]] name = "pallet-beefy-mmr" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "binary-merkle-tree", @@ -5306,7 +5318,7 @@ dependencies = [ [[package]] name = "pallet-bounties" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5324,7 +5336,7 @@ dependencies = [ [[package]] name = "pallet-child-bounties" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5343,7 +5355,7 @@ dependencies = [ [[package]] name = "pallet-collective" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5360,7 +5372,7 @@ dependencies = [ [[package]] name = "pallet-conviction-voting" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "assert_matches", "frame-benchmarking", @@ -5377,7 +5389,7 @@ dependencies = [ [[package]] name = "pallet-democracy" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5395,7 +5407,7 @@ dependencies = [ [[package]] name = "pallet-election-provider-multi-phase" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5418,7 +5430,7 @@ dependencies = [ [[package]] name = "pallet-election-provider-support-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5431,7 +5443,7 @@ dependencies = [ [[package]] name = "pallet-elections-phragmen" version = "5.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5449,7 +5461,7 @@ dependencies = [ [[package]] name = "pallet-fast-unstake" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5467,7 +5479,7 @@ dependencies = [ [[package]] name = "pallet-grandpa" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5490,7 +5502,7 @@ dependencies = [ [[package]] name = "pallet-identity" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "enumflags2", "frame-benchmarking", @@ -5506,7 +5518,7 @@ dependencies = [ [[package]] name = "pallet-im-online" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5526,7 +5538,7 @@ dependencies = [ [[package]] name = "pallet-indices" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5543,7 +5555,7 @@ dependencies = [ [[package]] name = "pallet-membership" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5560,7 +5572,7 @@ dependencies = [ [[package]] name = "pallet-mmr" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5577,7 +5589,7 @@ dependencies = [ [[package]] name = "pallet-multisig" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5593,7 +5605,7 @@ dependencies = [ [[package]] name = "pallet-nis" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5609,7 +5621,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools" version = "1.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5626,7 +5638,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools-benchmarking" version = "1.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5646,7 +5658,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools-runtime-api" version = "1.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "pallet-nomination-pools", "parity-scale-codec", @@ -5657,7 +5669,7 @@ dependencies = [ [[package]] name = "pallet-offences" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5674,7 +5686,7 @@ dependencies = [ [[package]] name = "pallet-offences-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5698,7 +5710,7 @@ dependencies = [ [[package]] name = "pallet-preimage" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5715,7 +5727,7 @@ dependencies = [ [[package]] name = "pallet-proxy" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5730,7 +5742,7 @@ dependencies = [ [[package]] name = "pallet-ranked-collective" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5748,7 +5760,7 @@ dependencies = [ [[package]] name = "pallet-recovery" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5763,7 +5775,7 @@ dependencies = [ [[package]] name = "pallet-referenda" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "assert_matches", "frame-benchmarking", @@ -5782,7 +5794,7 @@ dependencies = [ [[package]] name = "pallet-scheduler" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5799,7 +5811,7 @@ dependencies = [ [[package]] name = "pallet-session" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5820,7 +5832,7 @@ dependencies = [ [[package]] name = "pallet-session-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5836,7 +5848,7 @@ dependencies = [ [[package]] name = "pallet-society" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5850,7 +5862,7 @@ dependencies = [ [[package]] name = "pallet-staking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -5873,7 +5885,7 @@ dependencies = [ [[package]] name = "pallet-staking-reward-curve" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro-crate", "proc-macro2", @@ -5884,7 +5896,7 @@ dependencies = [ [[package]] name = "pallet-staking-reward-fn" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "log", "sp-arithmetic", @@ -5893,7 +5905,7 @@ dependencies = [ [[package]] name = "pallet-staking-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "sp-api", @@ -5902,7 +5914,7 @@ dependencies = [ [[package]] name = "pallet-state-trie-migration" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5919,7 +5931,7 @@ dependencies = [ [[package]] name = "pallet-sudo" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5933,7 +5945,7 @@ dependencies = [ [[package]] name = "pallet-timestamp" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5951,7 +5963,7 @@ dependencies = [ [[package]] name = "pallet-tips" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -5970,7 +5982,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-support", "frame-system", @@ -5986,7 +5998,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment-rpc" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "jsonrpsee", "pallet-transaction-payment-rpc-runtime-api", @@ -6002,7 +6014,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment-rpc-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "pallet-transaction-payment", "parity-scale-codec", @@ -6014,7 +6026,7 @@ dependencies = [ [[package]] name = "pallet-treasury" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -6031,7 +6043,7 @@ dependencies = [ [[package]] name = "pallet-uniques" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -6046,7 +6058,7 @@ dependencies = [ [[package]] name = "pallet-utility" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -6062,7 +6074,7 @@ dependencies = [ [[package]] name = "pallet-vesting" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -6077,7 +6089,7 @@ dependencies = [ [[package]] name = "pallet-whitelist" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-benchmarking", "frame-support", @@ -6091,7 +6103,7 @@ dependencies = [ [[package]] name = "pallet-xcm" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bounded-collections", "frame-benchmarking", @@ -6115,7 +6127,7 @@ dependencies = [ [[package]] name = "pallet-xcm-benchmarks" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-benchmarking", "frame-support", @@ -6420,7 +6432,7 @@ checksum = "e8d0eef3571242013a0d5dc84861c3ae4a652e56e12adf8bdc26ff5f8cb34c94" [[package]] name = "polkadot" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_cmd", "color-eyre", @@ -6437,7 +6449,7 @@ dependencies = [ [[package]] name = "polkadot-approval-distribution" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "env_logger 0.9.0", @@ -6462,7 +6474,7 @@ dependencies = [ [[package]] name = "polkadot-availability-bitfield-distribution" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "bitvec", @@ -6487,7 +6499,7 @@ dependencies = [ [[package]] name = "polkadot-availability-distribution" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "derive_more", @@ -6516,7 +6528,7 @@ dependencies = [ [[package]] name = "polkadot-availability-recovery" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "env_logger 0.9.0", @@ -6545,7 +6557,7 @@ dependencies = [ [[package]] name = "polkadot-cli" -version = "0.9.37" +version = "0.9.39" dependencies = [ "clap 4.0.15", "frame-benchmarking-cli", @@ -6573,7 +6585,7 @@ dependencies = [ [[package]] name = "polkadot-client" -version = "0.9.37" +version = "0.9.39" dependencies = [ "async-trait", "frame-benchmarking", @@ -6617,7 +6629,7 @@ dependencies = [ [[package]] name = "polkadot-collator-protocol" -version = "0.9.37" +version = "0.9.39" dependencies = [ "always-assert", "assert_matches", @@ -6646,7 +6658,7 @@ dependencies = [ [[package]] name = "polkadot-core-primitives" -version = "0.9.37" +version = "0.9.39" dependencies = [ "parity-scale-codec", "scale-info", @@ -6657,7 +6669,7 @@ dependencies = [ [[package]] name = "polkadot-dispute-distribution" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -6689,7 +6701,7 @@ dependencies = [ [[package]] name = "polkadot-erasure-coding" -version = "0.9.37" +version = "0.9.39" dependencies = [ "criterion", "parity-scale-codec", @@ -6703,7 +6715,7 @@ dependencies = [ [[package]] name = "polkadot-gossip-support" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -6730,7 +6742,7 @@ dependencies = [ [[package]] name = "polkadot-network-bridge" -version = "0.9.37" +version = "0.9.39" dependencies = [ "always-assert", "assert_matches", @@ -6750,7 +6762,6 @@ dependencies = [ "polkadot-primitives", "polkadot-primitives-test-helpers", "sc-network", - "sc-network-common", "sp-consensus", "sp-core", "sp-keyring", @@ -6760,7 +6771,7 @@ dependencies = [ [[package]] name = "polkadot-node-collation-generation" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures", "parity-scale-codec", @@ -6779,7 +6790,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-approval-voting" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -6818,7 +6829,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-av-store" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "bitvec", @@ -6847,7 +6858,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-backing" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "bitvec", @@ -6873,7 +6884,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-bitfield-signing" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures", "polkadot-node-subsystem", @@ -6889,7 +6900,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-candidate-validation" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -6913,7 +6924,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-chain-api" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures", "maplit", @@ -6932,7 +6943,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-chain-selection" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "futures", @@ -6953,7 +6964,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-dispute-coordinator" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "fatality", @@ -6981,7 +6992,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-parachains-inherent" -version = "0.9.37" +version = "0.9.39" dependencies = [ "async-trait", "futures", @@ -6997,7 +7008,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-provisioner" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "fatality", @@ -7018,7 +7029,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf" -version = "0.9.37" +version = "0.9.39" dependencies = [ "always-assert", "assert_matches", @@ -7057,7 +7068,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf-checker" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures", "futures-timer", @@ -7080,7 +7091,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-runtime-api" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures", "lru 0.9.0", @@ -7101,7 +7112,7 @@ dependencies = [ [[package]] name = "polkadot-node-jaeger" -version = "0.9.37" +version = "0.9.39" dependencies = [ "lazy_static", "log", @@ -7118,7 +7129,7 @@ dependencies = [ [[package]] name = "polkadot-node-metrics" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_cmd", "bs58", @@ -7145,7 +7156,7 @@ dependencies = [ [[package]] name = "polkadot-node-network-protocol" -version = "0.9.37" +version = "0.9.39" dependencies = [ "async-trait", "derive_more", @@ -7160,7 +7171,6 @@ dependencies = [ "rand_chacha 0.3.1", "sc-authority-discovery", "sc-network", - "sc-network-common", "strum", "thiserror", "tracing-gum", @@ -7168,7 +7178,7 @@ dependencies = [ [[package]] name = "polkadot-node-primitives" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bounded-vec", "futures", @@ -7191,7 +7201,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem" -version = "0.9.37" +version = "0.9.39" dependencies = [ "polkadot-node-jaeger", "polkadot-node-subsystem-types", @@ -7200,7 +7210,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-test-helpers" -version = "0.9.37" +version = "0.9.39" dependencies = [ "async-trait", "futures", @@ -7218,7 +7228,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-types" -version = "0.9.37" +version = "0.9.39" dependencies = [ "async-trait", "derive_more", @@ -7240,7 +7250,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-util" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -7281,7 +7291,7 @@ dependencies = [ [[package]] name = "polkadot-overseer" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -7307,7 +7317,7 @@ dependencies = [ [[package]] name = "polkadot-parachain" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bounded-collections", "derive_more", @@ -7323,7 +7333,7 @@ dependencies = [ [[package]] name = "polkadot-performance-test" -version = "0.9.37" +version = "0.9.39" dependencies = [ "env_logger 0.9.0", "kusama-runtime", @@ -7338,7 +7348,7 @@ dependencies = [ [[package]] name = "polkadot-primitives" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "hex-literal", @@ -7363,7 +7373,7 @@ dependencies = [ [[package]] name = "polkadot-primitives-test-helpers" -version = "0.9.37" +version = "0.9.39" dependencies = [ "polkadot-primitives", "rand 0.8.5", @@ -7375,7 +7385,7 @@ dependencies = [ [[package]] name = "polkadot-rpc" -version = "0.9.37" +version = "0.9.39" dependencies = [ "jsonrpsee", "mmr-rpc", @@ -7406,7 +7416,7 @@ dependencies = [ [[package]] name = "polkadot-runtime" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "frame-benchmarking", @@ -7503,7 +7513,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-common" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "frame-benchmarking", @@ -7553,7 +7563,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-constants" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "polkadot-primitives", @@ -7566,7 +7576,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-metrics" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bs58", "parity-scale-codec", @@ -7577,7 +7587,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-parachains" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "bitflags", @@ -7630,7 +7640,7 @@ dependencies = [ [[package]] name = "polkadot-service" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -7746,7 +7756,7 @@ dependencies = [ [[package]] name = "polkadot-statement-distribution" -version = "0.9.37" +version = "0.9.39" dependencies = [ "arrayvec 0.5.2", "assert_matches", @@ -7777,7 +7787,7 @@ dependencies = [ [[package]] name = "polkadot-statement-table" -version = "0.9.37" +version = "0.9.39" dependencies = [ "parity-scale-codec", "polkadot-primitives", @@ -7786,7 +7796,7 @@ dependencies = [ [[package]] name = "polkadot-test-client" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures", "parity-scale-codec", @@ -7812,7 +7822,7 @@ dependencies = [ [[package]] name = "polkadot-test-malus" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "async-trait", @@ -7840,7 +7850,7 @@ dependencies = [ [[package]] name = "polkadot-test-runtime" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "frame-election-provider-support", @@ -7905,7 +7915,7 @@ dependencies = [ [[package]] name = "polkadot-test-service" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-benchmarking", "frame-system", @@ -7934,7 +7944,6 @@ dependencies = [ "sc-consensus-grandpa", "sc-executor", "sc-network", - "sc-network-common", "sc-service", "sc-tracing", "sc-transaction-pool", @@ -7960,7 +7969,7 @@ dependencies = [ [[package]] name = "polkadot-voter-bags" -version = "0.9.37" +version = "0.9.39" dependencies = [ "clap 4.0.15", "generate-bags", @@ -8100,10 +8109,11 @@ dependencies = [ [[package]] name = "prioritized-metered-channel" -version = "0.2.0" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "382698e48a268c832d0b181ed438374a6bb708a82a8ca273bb0f61c74cf209c4" +checksum = "a3caef72a78ca8e77cbdfa87dd516ebb79d4cbe5b42e3b8435b463a8261339ff" dependencies = [ + "async-channel", "coarsetime", "crossbeam-queue", "derive_more", @@ -8437,26 +8447,23 @@ checksum = "60a357793950651c4ed0f3f52338f53b2f809f32d83a07f72909fa13e4c6c1e3" [[package]] name = "rayon" -version = "1.5.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c06aca804d41dbc8ba42dfd964f0d01334eceb64314b9ecf7c5fad5188a06d90" +checksum = "1d2df5196e37bcc87abebc0053e20787d73847bb33134a69841207dd0a47f03b" dependencies = [ - "autocfg", - "crossbeam-deque", "either", "rayon-core", ] [[package]] name = "rayon-core" -version = "1.9.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d78120e2c850279833f1dd3582f730c4ab53ed95aeaaaa862a2a5c71b1656d8e" +checksum = "4b8f95bd6966f5c87776639160a66bd8ab9895d9d4ab01ddba9fc60661aebe8d" dependencies = [ "crossbeam-channel", "crossbeam-deque", "crossbeam-utils", - "lazy_static", "num_cpus", ] @@ -8589,7 +8596,7 @@ dependencies = [ [[package]] name = "remote-ext-tests-bags-list" -version = "0.9.37" +version = "0.9.39" dependencies = [ "clap 4.0.15", "frame-system", @@ -8700,7 +8707,7 @@ dependencies = [ [[package]] name = "rococo-runtime" -version = "0.9.37" +version = "0.9.39" dependencies = [ "binary-merkle-tree", "frame-benchmarking", @@ -8793,7 +8800,7 @@ dependencies = [ [[package]] name = "rococo-runtime-constants" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "polkadot-primitives", @@ -9008,7 +9015,7 @@ dependencies = [ [[package]] name = "sc-allocator" version = "4.1.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "log", "sp-core", @@ -9019,7 +9026,7 @@ dependencies = [ [[package]] name = "sc-authority-discovery" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -9032,6 +9039,7 @@ dependencies = [ "prost-build", "rand 0.8.5", "sc-client-api", + "sc-network", "sc-network-common", "sp-api", "sp-authority-discovery", @@ -9046,7 +9054,7 @@ dependencies = [ [[package]] name = "sc-basic-authorship" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "futures-timer", @@ -9069,7 +9077,7 @@ dependencies = [ [[package]] name = "sc-block-builder" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "sc-client-api", @@ -9084,13 +9092,13 @@ dependencies = [ [[package]] name = "sc-chain-spec" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "memmap2", "sc-chain-spec-derive", "sc-client-api", "sc-executor", - "sc-network-common", + "sc-network", "sc-telemetry", "serde", "serde_json", @@ -9103,7 +9111,7 @@ dependencies = [ [[package]] name = "sc-chain-spec-derive" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro-crate", "proc-macro2", @@ -9114,7 +9122,7 @@ dependencies = [ [[package]] name = "sc-cli" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "chrono", @@ -9154,7 +9162,7 @@ dependencies = [ [[package]] name = "sc-client-api" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "fnv", "futures", @@ -9180,7 +9188,7 @@ dependencies = [ [[package]] name = "sc-client-db" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "hash-db", "kvdb", @@ -9206,7 +9214,7 @@ dependencies = [ [[package]] name = "sc-consensus" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -9231,7 +9239,7 @@ dependencies = [ [[package]] name = "sc-consensus-babe" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "fork-tree", @@ -9270,7 +9278,7 @@ dependencies = [ [[package]] name = "sc-consensus-babe-rpc" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "jsonrpsee", @@ -9292,7 +9300,7 @@ dependencies = [ [[package]] name = "sc-consensus-beefy" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "async-trait", @@ -9327,7 +9335,7 @@ dependencies = [ [[package]] name = "sc-consensus-beefy-rpc" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "jsonrpsee", @@ -9346,7 +9354,7 @@ dependencies = [ [[package]] name = "sc-consensus-epochs" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "fork-tree", "parity-scale-codec", @@ -9359,7 +9367,7 @@ dependencies = [ [[package]] name = "sc-consensus-grandpa" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ahash 0.8.2", "array-bytes", @@ -9399,7 +9407,7 @@ dependencies = [ [[package]] name = "sc-consensus-grandpa-rpc" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "finality-grandpa", "futures", @@ -9419,7 +9427,7 @@ dependencies = [ [[package]] name = "sc-consensus-slots" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -9442,7 +9450,7 @@ dependencies = [ [[package]] name = "sc-executor" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "lru 0.8.1", "parity-scale-codec", @@ -9466,7 +9474,7 @@ dependencies = [ [[package]] name = "sc-executor-common" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "sc-allocator", "sp-maybe-compressed-blob", @@ -9479,7 +9487,7 @@ dependencies = [ [[package]] name = "sc-executor-wasmi" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "log", "sc-allocator", @@ -9492,7 +9500,7 @@ dependencies = [ [[package]] name = "sc-executor-wasmtime" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "anyhow", "cfg-if", @@ -9510,13 +9518,14 @@ dependencies = [ [[package]] name = "sc-informant" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ansi_term", "futures", "futures-timer", "log", "sc-client-api", + "sc-network", "sc-network-common", "sp-blockchain", "sp-runtime", @@ -9525,7 +9534,7 @@ dependencies = [ [[package]] name = "sc-keystore" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "async-trait", @@ -9540,7 +9549,7 @@ dependencies = [ [[package]] name = "sc-network" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "async-channel", @@ -9553,6 +9562,7 @@ dependencies = [ "futures-timer", "ip_network", "libp2p", + "linked_hash_set", "log", "lru 0.8.1", "mockall", @@ -9583,7 +9593,7 @@ dependencies = [ [[package]] name = "sc-network-bitswap" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "cid", "futures", @@ -9592,6 +9602,7 @@ dependencies = [ "prost", "prost-build", "sc-client-api", + "sc-network", "sc-network-common", "sp-blockchain", "sp-runtime", @@ -9602,7 +9613,7 @@ dependencies = [ [[package]] name = "sc-network-common" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "async-trait", @@ -9611,7 +9622,6 @@ dependencies = [ "futures", "futures-timer", "libp2p", - "linked_hash_set", "parity-scale-codec", "prost-build", "sc-consensus", @@ -9631,7 +9641,7 @@ dependencies = [ [[package]] name = "sc-network-gossip" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ahash 0.8.2", "futures", @@ -9639,6 +9649,7 @@ dependencies = [ "libp2p", "log", "lru 0.8.1", + "sc-network", "sc-network-common", "sc-peerset", "sp-runtime", @@ -9649,7 +9660,7 @@ dependencies = [ [[package]] name = "sc-network-light" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "futures", @@ -9659,6 +9670,7 @@ dependencies = [ "prost", "prost-build", "sc-client-api", + "sc-network", "sc-network-common", "sc-peerset", "sp-blockchain", @@ -9670,7 +9682,7 @@ dependencies = [ [[package]] name = "sc-network-sync" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "async-trait", @@ -9686,6 +9698,7 @@ dependencies = [ "prost-build", "sc-client-api", "sc-consensus", + "sc-network", "sc-network-common", "sc-peerset", "sc-utils", @@ -9703,7 +9716,7 @@ dependencies = [ [[package]] name = "sc-network-transactions" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "futures", @@ -9711,6 +9724,7 @@ dependencies = [ "log", "parity-scale-codec", "pin-project", + "sc-network", "sc-network-common", "sc-peerset", "sc-utils", @@ -9722,7 +9736,7 @@ dependencies = [ [[package]] name = "sc-offchain" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "bytes", @@ -9738,6 +9752,7 @@ dependencies = [ "parking_lot 0.12.1", "rand 0.8.5", "sc-client-api", + "sc-network", "sc-network-common", "sc-peerset", "sc-utils", @@ -9752,7 +9767,7 @@ dependencies = [ [[package]] name = "sc-peerset" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "libp2p", @@ -9765,7 +9780,7 @@ dependencies = [ [[package]] name = "sc-proposer-metrics" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "log", "substrate-prometheus-endpoint", @@ -9774,7 +9789,7 @@ dependencies = [ [[package]] name = "sc-rpc" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "jsonrpsee", @@ -9804,7 +9819,7 @@ dependencies = [ [[package]] name = "sc-rpc-api" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -9823,7 +9838,7 @@ dependencies = [ [[package]] name = "sc-rpc-server" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "http", "jsonrpsee", @@ -9838,7 +9853,7 @@ dependencies = [ [[package]] name = "sc-rpc-spec-v2" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "futures", @@ -9864,7 +9879,7 @@ dependencies = [ [[package]] name = "sc-service" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "directories", @@ -9930,7 +9945,7 @@ dependencies = [ [[package]] name = "sc-state-db" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "log", "parity-scale-codec", @@ -9941,7 +9956,7 @@ dependencies = [ [[package]] name = "sc-storage-monitor" version = "0.1.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "clap 4.0.15", "fs4", @@ -9957,7 +9972,7 @@ dependencies = [ [[package]] name = "sc-sync-state-rpc" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -9976,7 +9991,7 @@ dependencies = [ [[package]] name = "sc-sysinfo" version = "6.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "libc", @@ -9995,7 +10010,7 @@ dependencies = [ [[package]] name = "sc-telemetry" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "chrono", "futures", @@ -10014,7 +10029,7 @@ dependencies = [ [[package]] name = "sc-tracing" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ansi_term", "atty", @@ -10045,7 +10060,7 @@ dependencies = [ [[package]] name = "sc-tracing-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro-crate", "proc-macro2", @@ -10056,7 +10071,7 @@ dependencies = [ [[package]] name = "sc-transaction-pool" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -10083,7 +10098,7 @@ dependencies = [ [[package]] name = "sc-transaction-pool-api" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -10097,7 +10112,7 @@ dependencies = [ [[package]] name = "sc-utils" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-channel", "futures", @@ -10535,7 +10550,7 @@ checksum = "03b634d87b960ab1a38c4fe143b508576f075e7c978bfad18217645ebfdfa2ec" [[package]] name = "slot-range-helper" -version = "0.9.37" +version = "0.9.39" dependencies = [ "enumn", "parity-scale-codec", @@ -10612,7 +10627,7 @@ dependencies = [ [[package]] name = "sp-api" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "hash-db", "log", @@ -10630,9 +10645,11 @@ dependencies = [ [[package]] name = "sp-api-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ + "Inflector", "blake2", + "expander 1.0.0", "proc-macro-crate", "proc-macro2", "quote", @@ -10642,7 +10659,7 @@ dependencies = [ [[package]] name = "sp-application-crypto" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -10655,7 +10672,7 @@ dependencies = [ [[package]] name = "sp-arithmetic" version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "integer-sqrt", "num-traits", @@ -10669,7 +10686,7 @@ dependencies = [ [[package]] name = "sp-authority-discovery" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -10682,7 +10699,7 @@ dependencies = [ [[package]] name = "sp-block-builder" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "sp-api", @@ -10694,7 +10711,7 @@ dependencies = [ [[package]] name = "sp-blockchain" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "log", @@ -10712,7 +10729,7 @@ dependencies = [ [[package]] name = "sp-consensus" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -10727,7 +10744,7 @@ dependencies = [ [[package]] name = "sp-consensus-aura" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "parity-scale-codec", @@ -10745,7 +10762,7 @@ dependencies = [ [[package]] name = "sp-consensus-babe" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "merlin", @@ -10768,7 +10785,7 @@ dependencies = [ [[package]] name = "sp-consensus-beefy" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "lazy_static", "parity-scale-codec", @@ -10787,7 +10804,7 @@ dependencies = [ [[package]] name = "sp-consensus-grandpa" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "finality-grandpa", "log", @@ -10805,7 +10822,7 @@ dependencies = [ [[package]] name = "sp-consensus-slots" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -10817,7 +10834,7 @@ dependencies = [ [[package]] name = "sp-consensus-vrf" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -10830,7 +10847,7 @@ dependencies = [ [[package]] name = "sp-core" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "base58", @@ -10873,7 +10890,7 @@ dependencies = [ [[package]] name = "sp-core-hashing" version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "blake2b_simd", "byteorder", @@ -10887,7 +10904,7 @@ dependencies = [ [[package]] name = "sp-core-hashing-proc-macro" version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro2", "quote", @@ -10898,7 +10915,7 @@ dependencies = [ [[package]] name = "sp-database" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "kvdb", "parking_lot 0.12.1", @@ -10907,7 +10924,7 @@ dependencies = [ [[package]] name = "sp-debug-derive" version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro2", "quote", @@ -10917,7 +10934,7 @@ dependencies = [ [[package]] name = "sp-externalities" version = "0.13.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "environmental", "parity-scale-codec", @@ -10928,7 +10945,7 @@ dependencies = [ [[package]] name = "sp-inherents" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "impl-trait-for-tuples", @@ -10943,7 +10960,7 @@ dependencies = [ [[package]] name = "sp-io" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "bytes", "ed25519", @@ -10968,7 +10985,7 @@ dependencies = [ [[package]] name = "sp-keyring" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "lazy_static", "sp-core", @@ -10979,7 +10996,7 @@ dependencies = [ [[package]] name = "sp-keystore" version = "0.13.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures", @@ -10996,7 +11013,7 @@ dependencies = [ [[package]] name = "sp-maybe-compressed-blob" version = "4.1.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "thiserror", "zstd", @@ -11005,7 +11022,7 @@ dependencies = [ [[package]] name = "sp-mmr-primitives" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ckb-merkle-mountain-range", "log", @@ -11023,7 +11040,7 @@ dependencies = [ [[package]] name = "sp-npos-elections" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -11037,7 +11054,7 @@ dependencies = [ [[package]] name = "sp-offchain" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "sp-api", "sp-core", @@ -11047,7 +11064,7 @@ dependencies = [ [[package]] name = "sp-panic-handler" version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "backtrace", "lazy_static", @@ -11057,7 +11074,7 @@ dependencies = [ [[package]] name = "sp-rpc" version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "rustc-hash", "serde", @@ -11067,7 +11084,7 @@ dependencies = [ [[package]] name = "sp-runtime" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "either", "hash256-std-hasher", @@ -11089,7 +11106,7 @@ dependencies = [ [[package]] name = "sp-runtime-interface" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "bytes", "impl-trait-for-tuples", @@ -11107,7 +11124,7 @@ dependencies = [ [[package]] name = "sp-runtime-interface-proc-macro" version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "Inflector", "proc-macro-crate", @@ -11119,7 +11136,7 @@ dependencies = [ [[package]] name = "sp-session" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -11133,7 +11150,7 @@ dependencies = [ [[package]] name = "sp-staking" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -11145,7 +11162,7 @@ dependencies = [ [[package]] name = "sp-state-machine" version = "0.13.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "hash-db", "log", @@ -11165,12 +11182,12 @@ dependencies = [ [[package]] name = "sp-std" version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" [[package]] name = "sp-storage" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "impl-serde", "parity-scale-codec", @@ -11183,7 +11200,7 @@ dependencies = [ [[package]] name = "sp-timestamp" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "futures-timer", @@ -11198,7 +11215,7 @@ dependencies = [ [[package]] name = "sp-tracing" version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "sp-std", @@ -11210,7 +11227,7 @@ dependencies = [ [[package]] name = "sp-transaction-pool" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "sp-api", "sp-runtime", @@ -11219,7 +11236,7 @@ dependencies = [ [[package]] name = "sp-transaction-storage-proof" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "log", @@ -11235,7 +11252,7 @@ dependencies = [ [[package]] name = "sp-trie" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ahash 0.8.2", "hash-db", @@ -11258,7 +11275,7 @@ dependencies = [ [[package]] name = "sp-version" version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "impl-serde", "parity-scale-codec", @@ -11275,7 +11292,7 @@ dependencies = [ [[package]] name = "sp-version-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "proc-macro2", @@ -11286,7 +11303,7 @@ dependencies = [ [[package]] name = "sp-wasm-interface" version = "7.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "anyhow", "impl-trait-for-tuples", @@ -11300,7 +11317,7 @@ dependencies = [ [[package]] name = "sp-weights" version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "parity-scale-codec", "scale-info", @@ -11351,7 +11368,7 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "staking-miner" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_cmd", "clap 4.0.15", @@ -11520,7 +11537,7 @@ dependencies = [ [[package]] name = "substrate-build-script-utils" version = "3.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "platforms", ] @@ -11528,7 +11545,7 @@ dependencies = [ [[package]] name = "substrate-frame-rpc-system" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "frame-system-rpc-runtime-api", "futures", @@ -11547,7 +11564,7 @@ dependencies = [ [[package]] name = "substrate-prometheus-endpoint" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "hyper", "log", @@ -11559,7 +11576,7 @@ dependencies = [ [[package]] name = "substrate-rpc-client" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "jsonrpsee", @@ -11572,7 +11589,7 @@ dependencies = [ [[package]] name = "substrate-state-trie-migration-rpc" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "jsonrpsee", "log", @@ -11591,7 +11608,7 @@ dependencies = [ [[package]] name = "substrate-test-client" version = "2.0.1" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "array-bytes", "async-trait", @@ -11617,7 +11634,7 @@ dependencies = [ [[package]] name = "substrate-test-utils" version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "futures", "substrate-test-utils-derive", @@ -11627,7 +11644,7 @@ dependencies = [ [[package]] name = "substrate-test-utils-derive" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "proc-macro-crate", "proc-macro2", @@ -11638,7 +11655,7 @@ dependencies = [ [[package]] name = "substrate-wasm-builder" version = "5.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "ansi_term", "build-helper", @@ -11786,7 +11803,7 @@ checksum = "13a4ec180a2de59b57434704ccfad967f789b12737738798fa08798cd5824c16" [[package]] name = "test-parachain-adder" -version = "0.9.37" +version = "0.9.39" dependencies = [ "dlmalloc", "parity-scale-codec", @@ -11799,7 +11816,7 @@ dependencies = [ [[package]] name = "test-parachain-adder-collator" -version = "0.9.37" +version = "0.9.39" dependencies = [ "clap 4.0.15", "futures", @@ -11825,14 +11842,14 @@ dependencies = [ [[package]] name = "test-parachain-halt" -version = "0.9.37" +version = "0.9.39" dependencies = [ "substrate-wasm-builder", ] [[package]] name = "test-parachain-undying" -version = "0.9.37" +version = "0.9.39" dependencies = [ "dlmalloc", "log", @@ -11846,7 +11863,7 @@ dependencies = [ [[package]] name = "test-parachain-undying-collator" -version = "0.9.37" +version = "0.9.39" dependencies = [ "clap 4.0.15", "futures", @@ -11872,7 +11889,7 @@ dependencies = [ [[package]] name = "test-parachains" -version = "0.9.37" +version = "0.9.39" dependencies = [ "parity-scale-codec", "sp-core", @@ -11883,7 +11900,7 @@ dependencies = [ [[package]] name = "test-runtime-constants" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "polkadot-primitives", @@ -12280,7 +12297,7 @@ dependencies = [ [[package]] name = "tracing-gum" -version = "0.9.37" +version = "0.9.39" dependencies = [ "polkadot-node-jaeger", "polkadot-primitives", @@ -12290,7 +12307,7 @@ dependencies = [ [[package]] name = "tracing-gum-proc-macro" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "expander 0.0.6", @@ -12364,9 +12381,9 @@ dependencies = [ [[package]] name = "trie-db" -version = "0.26.0" +version = "0.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879380c0061b165ba1f036325b7f3478bc1a947814d9fc36d22c5d8e960b11bd" +checksum = "634d75c77ea43f2ad8ea9d9c58de49dfc9c3995bdef32b503df7883ff054e7f1" dependencies = [ "hash-db", "hashbrown 0.13.2", @@ -12377,9 +12394,9 @@ dependencies = [ [[package]] name = "trie-root" -version = "0.17.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a36c5ca3911ed3c9a5416ee6c679042064b93fc637ded67e25f92e68d783891" +checksum = "d4ed310ef5ab98f5fa467900ed906cb9232dd5376597e00fd4cba2a449d06c0b" dependencies = [ "hash-db", ] @@ -12439,7 +12456,7 @@ checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" [[package]] name = "try-runtime-cli" version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=master#310fe4095bff177ddcfa660ac0ab06171a25a9c2" +source = "git+https://github.com/paritytech/substrate?branch=master#ae83a672f2bbe6a4f7391f6a5b2b6fd7ad4f8651" dependencies = [ "async-trait", "clap 4.0.15", @@ -13381,7 +13398,7 @@ dependencies = [ [[package]] name = "westend-runtime" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bitvec", "frame-benchmarking", @@ -13478,7 +13495,7 @@ dependencies = [ [[package]] name = "westend-runtime-constants" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "polkadot-primitives", @@ -13778,7 +13795,7 @@ dependencies = [ [[package]] name = "xcm" -version = "0.9.37" +version = "0.9.39" dependencies = [ "bounded-collections", "derivative", @@ -13794,7 +13811,7 @@ dependencies = [ [[package]] name = "xcm-builder" -version = "0.9.37" +version = "0.9.39" dependencies = [ "assert_matches", "frame-support", @@ -13819,7 +13836,7 @@ dependencies = [ [[package]] name = "xcm-executor" -version = "0.9.37" +version = "0.9.39" dependencies = [ "environmental", "frame-benchmarking", @@ -13838,7 +13855,7 @@ dependencies = [ [[package]] name = "xcm-executor-integration-tests" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "frame-system", @@ -13858,7 +13875,7 @@ dependencies = [ [[package]] name = "xcm-procedural" -version = "0.9.37" +version = "0.9.39" dependencies = [ "Inflector", "proc-macro2", @@ -13868,7 +13885,7 @@ dependencies = [ [[package]] name = "xcm-simulator" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "parity-scale-codec", @@ -13884,7 +13901,7 @@ dependencies = [ [[package]] name = "xcm-simulator-example" -version = "0.9.37" +version = "0.9.39" dependencies = [ "frame-support", "frame-system", @@ -13910,7 +13927,7 @@ dependencies = [ [[package]] name = "xcm-simulator-fuzzer" -version = "0.9.37" +version = "0.9.39" dependencies = [ "arbitrary", "frame-support", @@ -13979,7 +13996,7 @@ dependencies = [ [[package]] name = "zombienet-backchannel" -version = "0.9.37" +version = "0.9.39" dependencies = [ "futures-util", "lazy_static", diff --git a/Cargo.toml b/Cargo.toml index db341c55315e..5ec0a74d5cac 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,7 @@ version.workspace = true authors = ["Parity Technologies "] edition = "2021" repository = "https://github.com/paritytech/polkadot.git" -version = "0.9.37" +version = "0.9.39" [dependencies] color-eyre = { version = "0.6.1", default-features = false } diff --git a/node/core/candidate-validation/src/lib.rs b/node/core/candidate-validation/src/lib.rs index fdd6ff002dd1..7d3fa964eeff 100644 --- a/node/core/candidate-validation/src/lib.rs +++ b/node/core/candidate-validation/src/lib.rs @@ -42,7 +42,7 @@ use polkadot_node_subsystem::{ use polkadot_node_subsystem_util::executor_params_at_relay_parent; use polkadot_parachain::primitives::{ValidationParams, ValidationResult as WasmValidationResult}; use polkadot_primitives::{ - vstaging::ExecutorParams, CandidateCommitments, CandidateDescriptor, CandidateReceipt, Hash, + CandidateCommitments, CandidateDescriptor, CandidateReceipt, ExecutorParams, Hash, OccupiedCoreAssumption, PersistedValidationData, PvfExecTimeoutKind, PvfPrepTimeoutKind, ValidationCode, ValidationCodeHash, }; @@ -623,7 +623,7 @@ where .await; if let Err(ref error) = result { - gum::info!(target: LOG_TARGET, ?para_id, ?error, "Failed to validate candidate",); + gum::info!(target: LOG_TARGET, ?para_id, ?error, "Failed to validate candidate"); } match result { @@ -638,9 +638,16 @@ where ))), Err(ValidationError::InvalidCandidate(WasmInvalidCandidate::PrepareError(e))) => { // In principle if preparation of the `WASM` fails, the current candidate can not be the - // reason for that. So we can't say whether it is invalid or not in addition with + // reason for that. So we can't say whether it is invalid or not. In addition, with // pre-checking enabled only valid runtimes should ever get enacted, so we can be - // reasonably sure that this is some local problem on the current node. + // reasonably sure that this is some local problem on the current node. However, as this + // particular error *seems* to indicate a deterministic error, we raise a warning. + gum::warn!( + target: LOG_TARGET, + ?para_id, + ?e, + "Deterministic error occurred during preparation (should have been ruled out by pre-checking phase)", + ); Err(ValidationFailed(e)) }, Ok(res) => diff --git a/node/core/pvf/Cargo.toml b/node/core/pvf/Cargo.toml index 3f88876283b7..479c3ba539c5 100644 --- a/node/core/pvf/Cargo.toml +++ b/node/core/pvf/Cargo.toml @@ -17,7 +17,7 @@ futures-timer = "3.0.2" gum = { package = "tracing-gum", path = "../../gum" } pin-project = "1.0.9" rand = "0.8.5" -rayon = "1.5.1" +rayon = "1.6.1" slotmap = "1.0" tempfile = "3.3.0" tikv-jemalloc-ctl = { version = "0.5.0", optional = true } diff --git a/node/core/pvf/src/artifacts.rs b/node/core/pvf/src/artifacts.rs index 7ddbaef38c5a..760e7ed46a1c 100644 --- a/node/core/pvf/src/artifacts.rs +++ b/node/core/pvf/src/artifacts.rs @@ -17,7 +17,7 @@ use crate::{error::PrepareError, host::PrepareResultSender, prepare::PrepareStats}; use always_assert::always; use polkadot_parachain::primitives::ValidationCodeHash; -use polkadot_primitives::vstaging::ExecutorParamsHash; +use polkadot_primitives::ExecutorParamsHash; use std::{ collections::HashMap, path::{Path, PathBuf}, @@ -219,7 +219,7 @@ impl Artifacts { #[cfg(test)] mod tests { use super::{ArtifactId, Artifacts}; - use polkadot_primitives::vstaging::ExecutorParamsHash; + use polkadot_primitives::ExecutorParamsHash; use sp_core::H256; use std::{path::Path, str::FromStr}; diff --git a/node/core/pvf/src/error.rs b/node/core/pvf/src/error.rs index 3f642cd6ed24..5edf435e7190 100644 --- a/node/core/pvf/src/error.rs +++ b/node/core/pvf/src/error.rs @@ -120,12 +120,6 @@ impl From for ValidationError { fn from(error: PrepareError) -> Self { // Here we need to classify the errors into two errors: deterministic and non-deterministic. // See [`PrepareError::is_deterministic`]. - // - // We treat the deterministic errors as `InvalidCandidate`. Should those occur they could - // potentially trigger disputes. - // - // All non-deterministic errors are qualified as `InternalError`s and will not trigger - // disputes. if error.is_deterministic() { ValidationError::InvalidCandidate(InvalidCandidate::PrepareError(error.to_string())) } else { diff --git a/node/core/pvf/src/execute/queue.rs b/node/core/pvf/src/execute/queue.rs index 9a89c46e7a14..af9c0d8672ef 100644 --- a/node/core/pvf/src/execute/queue.rs +++ b/node/core/pvf/src/execute/queue.rs @@ -30,7 +30,7 @@ use futures::{ stream::{FuturesUnordered, StreamExt as _}, Future, FutureExt, }; -use polkadot_primitives::vstaging::{ExecutorParams, ExecutorParamsHash}; +use polkadot_primitives::{ExecutorParams, ExecutorParamsHash}; use slotmap::HopSlotMap; use std::{ collections::VecDeque, diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 81e2228026f4..47a2e65af95a 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -30,7 +30,7 @@ use futures_timer::Delay; use parity_scale_codec::{Decode, Encode}; use polkadot_parachain::primitives::ValidationResult; -use polkadot_primitives::vstaging::ExecutorParams; +use polkadot_primitives::ExecutorParams; use std::{ path::{Path, PathBuf}, sync::{mpsc::channel, Arc}, diff --git a/node/core/pvf/src/executor_intf.rs b/node/core/pvf/src/executor_intf.rs index a1b5f9a4886c..af552077bc43 100644 --- a/node/core/pvf/src/executor_intf.rs +++ b/node/core/pvf/src/executor_intf.rs @@ -16,7 +16,7 @@ //! Interface to the Substrate Executor -use polkadot_primitives::vstaging::executor_params::{ExecutorParam, ExecutorParams}; +use polkadot_primitives::{ExecutorParam, ExecutorParams}; use sc_executor_common::{ runtime_blob::RuntimeBlob, wasm_runtime::{HeapAllocStrategy, InvokeMethod, WasmModule as _}, diff --git a/node/core/pvf/src/host.rs b/node/core/pvf/src/host.rs index d4071064d41a..eeb0936e898f 100644 --- a/node/core/pvf/src/host.rs +++ b/node/core/pvf/src/host.rs @@ -33,7 +33,7 @@ use futures::{ Future, FutureExt, SinkExt, StreamExt, }; use polkadot_parachain::primitives::ValidationResult; -use polkadot_primitives::vstaging::ExecutorParams; +use polkadot_primitives::ExecutorParams; use std::{ collections::HashMap, path::{Path, PathBuf}, diff --git a/node/core/pvf/src/pvf.rs b/node/core/pvf/src/pvf.rs index 3a40b972f09a..7e7aa6be5245 100644 --- a/node/core/pvf/src/pvf.rs +++ b/node/core/pvf/src/pvf.rs @@ -17,7 +17,7 @@ use crate::artifacts::ArtifactId; use parity_scale_codec::{Decode, Encode}; use polkadot_parachain::primitives::ValidationCodeHash; -use polkadot_primitives::vstaging::ExecutorParams; +use polkadot_primitives::ExecutorParams; use sp_core::blake2_256; use std::{ cmp::{Eq, PartialEq}, diff --git a/node/core/pvf/src/testing.rs b/node/core/pvf/src/testing.rs index 0370349e433d..52c208706bb6 100644 --- a/node/core/pvf/src/testing.rs +++ b/node/core/pvf/src/testing.rs @@ -19,7 +19,7 @@ //! N.B. This is not guarded with some feature flag. Overexposing items here may affect the final //! artifact even for production builds. -use polkadot_primitives::vstaging::ExecutorParams; +use polkadot_primitives::ExecutorParams; pub mod worker_common { pub use crate::worker_common::{spawn_with_program_path, SpawnErr}; diff --git a/node/core/pvf/tests/it/main.rs b/node/core/pvf/tests/it/main.rs index c35a638ba43b..6833a7de9fb0 100644 --- a/node/core/pvf/tests/it/main.rs +++ b/node/core/pvf/tests/it/main.rs @@ -21,7 +21,7 @@ use polkadot_node_core_pvf::{ JOB_TIMEOUT_WALL_CLOCK_FACTOR, }; use polkadot_parachain::primitives::{BlockData, ValidationParams, ValidationResult}; -use polkadot_primitives::vstaging::{ExecutorParam, ExecutorParams}; +use polkadot_primitives::{ExecutorParam, ExecutorParams}; use std::time::Duration; use tokio::sync::Mutex; diff --git a/node/core/runtime-api/src/cache.rs b/node/core/runtime-api/src/cache.rs index 82d2e0dbc8b3..63274f10c4bf 100644 --- a/node/core/runtime-api/src/cache.rs +++ b/node/core/runtime-api/src/cache.rs @@ -20,12 +20,11 @@ use lru::LruCache; use sp_consensus_babe::Epoch; use polkadot_primitives::{ - vstaging::ExecutorParams, AuthorityDiscoveryId, BlockNumber, CandidateCommitments, - CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, - GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, - OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, - SessionIndex, SessionInfo, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - ValidatorSignature, + AuthorityDiscoveryId, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, + Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, OccupiedCoreAssumption, + PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionIndex, SessionInfo, + ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, }; /// For consistency we have the same capacity for all caches. We use 128 as we'll only need that diff --git a/node/metrics/Cargo.toml b/node/metrics/Cargo.toml index 8a6938190887..63864c15a24d 100644 --- a/node/metrics/Cargo.toml +++ b/node/metrics/Cargo.toml @@ -10,7 +10,7 @@ futures = "0.3.21" futures-timer = "3.0.2" gum = { package = "tracing-gum", path = "../gum" } -metered = { package = "prioritized-metered-channel", version = "0.2.0" } +metered = { package = "prioritized-metered-channel", version = "0.4.0" } # Both `sc-service` and `sc-cli` are required by runtime metrics `logger_hook()`. sc-service = { git = "https://github.com/paritytech/substrate", branch = "master" } diff --git a/node/network/bridge/Cargo.toml b/node/network/bridge/Cargo.toml index a10983d4b1b1..a91918af2647 100644 --- a/node/network/bridge/Cargo.toml +++ b/node/network/bridge/Cargo.toml @@ -12,7 +12,6 @@ gum = { package = "tracing-gum", path = "../../gum" } polkadot-primitives = { path = "../../../primitives" } parity-scale-codec = { version = "3.3.0", default-features = false, features = ["derive"] } sc-network = { git = "https://github.com/paritytech/substrate", branch = "master" } -sc-network-common = { git = "https://github.com/paritytech/substrate", branch = "master" } sp-consensus = { git = "https://github.com/paritytech/substrate", branch = "master" } polkadot-node-metrics = { path = "../../metrics"} polkadot-node-network-protocol = { path = "../protocol" } diff --git a/node/network/bridge/src/network.rs b/node/network/bridge/src/network.rs index 28a84a19b321..3d598a181a07 100644 --- a/node/network/bridge/src/network.rs +++ b/node/network/bridge/src/network.rs @@ -22,13 +22,9 @@ use futures::{prelude::*, stream::BoxStream}; use parity_scale_codec::Encode; use sc_network::{ - multiaddr::Multiaddr, Event as NetworkEvent, IfDisconnected, NetworkService, OutboundFailure, - RequestFailure, -}; -use sc_network_common::{ - config::parse_addr, - protocol::ProtocolName, - service::{NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest}, + config::parse_addr, multiaddr::Multiaddr, types::ProtocolName, Event as NetworkEvent, + IfDisconnected, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, + NetworkService, OutboundFailure, RequestFailure, }; use polkadot_node_network_protocol::{ @@ -193,8 +189,9 @@ impl Network for Arc> { match pending_response .send(Err(RequestFailure::Network(OutboundFailure::DialFailure))) { - Err(_) => - gum::debug!(target: LOG_TARGET, "Sending failed request response failed."), + Err(_) => { + gum::debug!(target: LOG_TARGET, "Sending failed request response failed.") + }, Ok(_) => {}, } return diff --git a/node/network/protocol/Cargo.toml b/node/network/protocol/Cargo.toml index 112c00375c99..4b015619260f 100644 --- a/node/network/protocol/Cargo.toml +++ b/node/network/protocol/Cargo.toml @@ -13,7 +13,6 @@ polkadot-node-primitives = { path = "../../primitives" } polkadot-node-jaeger = { path = "../../jaeger" } parity-scale-codec = { version = "3.3.0", default-features = false, features = ["derive"] } sc-network = { git = "https://github.com/paritytech/substrate", branch = "master" } -sc-network-common = { git = "https://github.com/paritytech/substrate", branch = "master" } sc-authority-discovery = { git = "https://github.com/paritytech/substrate", branch = "master" } strum = { version = "0.24", features = ["derive"] } futures = "0.3.21" diff --git a/node/network/protocol/src/peer_set.rs b/node/network/protocol/src/peer_set.rs index faeea10e4cea..84c41051f753 100644 --- a/node/network/protocol/src/peer_set.rs +++ b/node/network/protocol/src/peer_set.rs @@ -18,9 +18,9 @@ use derive_more::Display; use polkadot_primitives::Hash; -use sc_network_common::{ +use sc_network::{ config::{NonDefaultSetConfig, SetConfig}, - protocol::ProtocolName, + types::ProtocolName, }; use std::{ collections::{hash_map::Entry, HashMap}, @@ -81,7 +81,7 @@ impl PeerSet { fallback_names, max_notification_size, handshake: None, - set_config: sc_network_common::config::SetConfig { + set_config: SetConfig { // we allow full nodes to connect to validators for gossip // to ensure any `MIN_GOSSIP_PEERS` always include reserved peers // we limit the amount of non-reserved slots to be less @@ -89,7 +89,7 @@ impl PeerSet { in_peers: super::MIN_GOSSIP_PEERS as u32 / 2 - 1, out_peers: super::MIN_GOSSIP_PEERS as u32 / 2 - 1, reserved_nodes: Vec::new(), - non_reserved_mode: sc_network_common::config::NonReservedPeerMode::Accept, + non_reserved_mode: sc_network::config::NonReservedPeerMode::Accept, }, }, PeerSet::Collation => NonDefaultSetConfig { @@ -103,9 +103,9 @@ impl PeerSet { out_peers: 0, reserved_nodes: Vec::new(), non_reserved_mode: if is_authority == IsAuthority::Yes { - sc_network_common::config::NonReservedPeerMode::Accept + sc_network::config::NonReservedPeerMode::Accept } else { - sc_network_common::config::NonReservedPeerMode::Deny + sc_network::config::NonReservedPeerMode::Deny }, }, }, @@ -190,7 +190,7 @@ impl IndexMut for PerPeerSet { pub fn peer_sets_info( is_authority: IsAuthority, peerset_protocol_names: &PeerSetProtocolNames, -) -> Vec { +) -> Vec { PeerSet::iter() .map(|s| s.get_info(is_authority, &peerset_protocol_names)) .collect() diff --git a/node/overseer/Cargo.toml b/node/overseer/Cargo.toml index 5d26c0b6e2bf..7f668ec20ad6 100644 --- a/node/overseer/Cargo.toml +++ b/node/overseer/Cargo.toml @@ -15,7 +15,7 @@ polkadot-node-primitives = { path = "../primitives" } polkadot-node-subsystem-types = { path = "../subsystem-types" } polkadot-node-metrics = { path = "../metrics" } polkadot-primitives = { path = "../../primitives" } -orchestra = "0.0.4" +orchestra = "0.2.0" gum = { package = "tracing-gum", path = "../gum" } lru = "0.9" sp-core = { git = "https://github.com/paritytech/substrate", branch = "master" } @@ -23,7 +23,7 @@ async-trait = "0.1.57" tikv-jemalloc-ctl = { version = "0.5.0", optional = true } [dev-dependencies] -metered = { package = "prioritized-metered-channel", version = "0.2.0" } +metered = { package = "prioritized-metered-channel", version = "0.4.0" } sp-core = { git = "https://github.com/paritytech/substrate", branch = "master" } futures = { version = "0.3.21", features = ["thread-pool"] } diff --git a/node/overseer/src/lib.rs b/node/overseer/src/lib.rs index 810f08af4857..f72426b1cb64 100644 --- a/node/overseer/src/lib.rs +++ b/node/overseer/src/lib.rs @@ -661,8 +661,9 @@ where ); metrics.memory_stats_snapshot(memory_stats_snapshot); }, - Err(e) => - gum::debug!(target: LOG_TARGET, "Failed to obtain memory stats: {:?}", e), + Err(e) => { + gum::debug!(target: LOG_TARGET, "Failed to obtain memory stats: {:?}", e) + }, }), Err(_) => { gum::debug!( diff --git a/node/overseer/src/tests.rs b/node/overseer/src/tests.rs index bc26402aedea..f461196606e1 100644 --- a/node/overseer/src/tests.rs +++ b/node/overseer/src/tests.rs @@ -1034,6 +1034,7 @@ fn overseer_all_subsystems_receive_signals_and_messages() { #[test] fn context_holds_onto_message_until_enough_signals_received() { + const CHANNEL_CAPACITY: usize = 1024; let (candidate_validation_bounded_tx, _) = metered::channel(CHANNEL_CAPACITY); let (candidate_backing_bounded_tx, _) = metered::channel(CHANNEL_CAPACITY); let (statement_distribution_bounded_tx, _) = metered::channel(CHANNEL_CAPACITY); diff --git a/node/service/src/lib.rs b/node/service/src/lib.rs index e8b56904ea8c..14f3c6ba1197 100644 --- a/node/service/src/lib.rs +++ b/node/service/src/lib.rs @@ -961,8 +961,7 @@ where let authority_discovery_service = if auth_or_collator || overseer_enable_anyways { use futures::StreamExt; - use sc_network::Event; - use sc_network_common::service::NetworkEventStream; + use sc_network::{Event, NetworkEventStream}; let authority_discovery_role = if role.is_authority() { sc_authority_discovery::Role::PublishAndDiscover(keystore_container.keystore()) diff --git a/node/service/src/overseer.rs b/node/service/src/overseer.rs index 1f12cdc05a96..39a57289a9f6 100644 --- a/node/service/src/overseer.rs +++ b/node/service/src/overseer.rs @@ -42,7 +42,7 @@ use polkadot_primitives::runtime_api::ParachainHost; use sc_authority_discovery::Service as AuthorityDiscoveryService; use sc_client_api::AuxStore; use sc_keystore::LocalKeystore; -use sc_network_common::service::NetworkStateInfo; +use sc_network::NetworkStateInfo; use sp_api::ProvideRuntimeApi; use sp_blockchain::HeaderBackend; use sp_consensus_babe::BabeApi; diff --git a/node/subsystem-types/Cargo.toml b/node/subsystem-types/Cargo.toml index 22528503ccc4..9ed095622cb5 100644 --- a/node/subsystem-types/Cargo.toml +++ b/node/subsystem-types/Cargo.toml @@ -13,7 +13,7 @@ polkadot-node-primitives = { path = "../primitives" } polkadot-node-network-protocol = { path = "../network/protocol" } polkadot-statement-table = { path = "../../statement-table" } polkadot-node-jaeger = { path = "../jaeger" } -orchestra = "0.0.4" +orchestra = "0.2.0" sc-network = { git = "https://github.com/paritytech/substrate", branch = "master" } sp-api = { git = "https://github.com/paritytech/substrate", branch = "master" } sp-consensus-babe = { git = "https://github.com/paritytech/substrate", branch = "master" } diff --git a/node/subsystem-types/src/messages.rs b/node/subsystem-types/src/messages.rs index e5b9db9e5e97..95895a5b0aec 100644 --- a/node/subsystem-types/src/messages.rs +++ b/node/subsystem-types/src/messages.rs @@ -39,9 +39,9 @@ use polkadot_node_primitives::{ SignedDisputeStatement, SignedFullStatement, ValidationResult, }; use polkadot_primitives::{ - vstaging::ExecutorParams, AuthorityDiscoveryId, BackedCandidate, BlockNumber, CandidateEvent, - CandidateHash, CandidateIndex, CandidateReceipt, CollatorId, CommittedCandidateReceipt, - CoreState, DisputeState, GroupIndex, GroupRotationInfo, Hash, Header as BlockHeader, + AuthorityDiscoveryId, BackedCandidate, BlockNumber, CandidateEvent, CandidateHash, + CandidateIndex, CandidateReceipt, CollatorId, CommittedCandidateReceipt, CoreState, + DisputeState, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, Header as BlockHeader, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, MultiDisputeStatementSet, OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, PvfExecTimeoutKind, SessionIndex, SessionInfo, SignedAvailabilityBitfield, SignedAvailabilityBitfields, diff --git a/node/subsystem-types/src/runtime_client.rs b/node/subsystem-types/src/runtime_client.rs index 81b33c62cc0a..7af3cb33696b 100644 --- a/node/subsystem-types/src/runtime_client.rs +++ b/node/subsystem-types/src/runtime_client.rs @@ -16,8 +16,8 @@ use async_trait::async_trait; use polkadot_primitives::{ - runtime_api::ParachainHost, vstaging::ExecutorParams, Block, BlockNumber, CandidateCommitments, - CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, + runtime_api::ParachainHost, Block, BlockNumber, CandidateCommitments, CandidateEvent, + CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, Id, InboundDownwardMessage, InboundHrmpMessage, OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionIndex, SessionInfo, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, diff --git a/node/subsystem-util/Cargo.toml b/node/subsystem-util/Cargo.toml index 9fd4af14cde1..20314afed06f 100644 --- a/node/subsystem-util/Cargo.toml +++ b/node/subsystem-util/Cargo.toml @@ -27,7 +27,7 @@ polkadot-node-network-protocol = { path = "../network/protocol" } polkadot-primitives = { path = "../../primitives" } polkadot-node-primitives = { path = "../primitives" } polkadot-overseer = { path = "../overseer" } -metered = { package = "prioritized-metered-channel", version = "0.2.0" } +metered = { package = "prioritized-metered-channel", version = "0.4.0" } sp-core = { git = "https://github.com/paritytech/substrate", branch = "master" } sp-application-crypto = { git = "https://github.com/paritytech/substrate", branch = "master" } diff --git a/node/subsystem-util/src/lib.rs b/node/subsystem-util/src/lib.rs index 3aafc14a7878..1fec56ef15a6 100644 --- a/node/subsystem-util/src/lib.rs +++ b/node/subsystem-util/src/lib.rs @@ -29,7 +29,7 @@ use polkadot_node_subsystem::{ messages::{RuntimeApiMessage, RuntimeApiRequest, RuntimeApiSender}, overseer, SubsystemSender, }; -use polkadot_primitives::vstaging::ExecutorParams; +use polkadot_primitives::ExecutorParams; pub use overseer::{ gen::{OrchestraError as OverseerError, Timeout}, diff --git a/node/test/performance-test/src/lib.rs b/node/test/performance-test/src/lib.rs index 4ba86014e1e4..636bb7a00ea9 100644 --- a/node/test/performance-test/src/lib.rs +++ b/node/test/performance-test/src/lib.rs @@ -18,7 +18,7 @@ use polkadot_erasure_coding::{obtain_chunks, reconstruct}; use polkadot_node_core_pvf::{sc_executor_common, sp_maybe_compressed_blob}; -use polkadot_primitives::vstaging::ExecutorParams; +use polkadot_primitives::ExecutorParams; use std::time::{Duration, Instant}; mod constants; diff --git a/node/test/service/Cargo.toml b/node/test/service/Cargo.toml index ec85fe4295e0..92a707bd9fca 100644 --- a/node/test/service/Cargo.toml +++ b/node/test/service/Cargo.toml @@ -45,7 +45,6 @@ sc-client-api = { git = "https://github.com/paritytech/substrate", branch = "mas sc-consensus = { git = "https://github.com/paritytech/substrate", branch = "master" } sc-executor = { git = "https://github.com/paritytech/substrate", branch = "master" } sc-network = { git = "https://github.com/paritytech/substrate", branch = "master" } -sc-network-common = { git = "https://github.com/paritytech/substrate", branch = "master" } sc-tracing = { git = "https://github.com/paritytech/substrate", branch = "master" } sc-transaction-pool = { git = "https://github.com/paritytech/substrate", branch = "master" } sc-service = { git = "https://github.com/paritytech/substrate", branch = "master", default-features = false } diff --git a/node/test/service/src/lib.rs b/node/test/service/src/lib.rs index 352959236ef0..f95ee9f75ed3 100644 --- a/node/test/service/src/lib.rs +++ b/node/test/service/src/lib.rs @@ -37,8 +37,10 @@ use polkadot_test_runtime::{ }; use sc_chain_spec::ChainSpec; use sc_client_api::execution_extensions::ExecutionStrategies; -use sc_network::{config::NetworkConfiguration, multiaddr}; -use sc_network_common::{config::TransportConfig, service::NetworkStateInfo}; +use sc_network::{ + config::{NetworkConfiguration, TransportConfig}, + multiaddr, NetworkStateInfo, +}; use sc_service::{ config::{ DatabaseSource, KeystoreConfig, MultiaddrWithPeerId, WasmExecutionMethod, diff --git a/primitives/src/lib.rs b/primitives/src/lib.rs index e242c7c0800c..c7c4b502d3f3 100644 --- a/primitives/src/lib.rs +++ b/primitives/src/lib.rs @@ -19,8 +19,8 @@ #![warn(missing_docs)] #![cfg_attr(not(feature = "std"), no_std)] -// `v2` is currently the latest stable version of the runtime API. -pub mod v2; +// `v4` is currently the latest stable version of the runtime API. +pub mod v4; // The 'staging' version is special - it contains primitives which are // still in development. Once they are considered stable, they will be @@ -33,7 +33,7 @@ pub mod runtime_api; // Current primitives not requiring versioning are exported here. // Primitives requiring versioning must not be exported and must be referred by an exact version. -pub use v2::{ +pub use v4::{ byzantine_threshold, check_candidate_backing, collator_signature_payload, metric_definitions, supermajority_threshold, well_known_keys, AbridgedHostConfiguration, AbridgedHrmpChannel, AccountId, AccountIndex, AccountPublic, ApprovalVote, AssignmentId, AuthorityDiscoveryId, @@ -42,10 +42,11 @@ pub use v2::{ CandidateReceipt, CheckedDisputeStatementSet, CheckedMultiDisputeStatementSet, CollatorId, CollatorSignature, CommittedCandidateReceipt, CompactStatement, ConsensusLog, CoreIndex, CoreOccupied, CoreState, DisputeState, DisputeStatement, DisputeStatementSet, DownwardMessage, - EncodeAs, ExplicitDisputeStatement, GroupIndex, GroupRotationInfo, Hash, HashT, HeadData, - Header, HrmpChannelId, Id, InboundDownwardMessage, InboundHrmpMessage, IndexedVec, - InherentData, InvalidDisputeStatementKind, Moment, MultiDisputeStatementSet, Nonce, - OccupiedCore, OccupiedCoreAssumption, OutboundHrmpMessage, ParathreadClaim, ParathreadEntry, + EncodeAs, ExecutorParam, ExecutorParams, ExecutorParamsHash, ExplicitDisputeStatement, + GroupIndex, GroupRotationInfo, Hash, HashT, HeadData, Header, HrmpChannelId, Id, + InboundDownwardMessage, InboundHrmpMessage, IndexedVec, InherentData, + InvalidDisputeStatementKind, Moment, MultiDisputeStatementSet, Nonce, OccupiedCore, + OccupiedCoreAssumption, OutboundHrmpMessage, ParathreadClaim, ParathreadEntry, PersistedValidationData, PvfCheckStatement, PvfExecTimeoutKind, PvfPrepTimeoutKind, RuntimeMetricLabel, RuntimeMetricLabelValue, RuntimeMetricLabelValues, RuntimeMetricLabels, RuntimeMetricOp, RuntimeMetricUpdate, ScheduledCore, ScrapedOnChainVotes, SessionIndex, @@ -59,4 +60,4 @@ pub use v2::{ }; #[cfg(feature = "std")] -pub use v2::{AssignmentPair, CollatorPair, ValidatorPair}; +pub use v4::{AssignmentPair, CollatorPair, ValidatorPair}; diff --git a/primitives/src/runtime_api.rs b/primitives/src/runtime_api.rs index 8275d595d61b..222eb9580ce0 100644 --- a/primitives/src/runtime_api.rs +++ b/primitives/src/runtime_api.rs @@ -111,8 +111,8 @@ //! from the stable primitives. use crate::{ - vstaging, BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, - CommittedCandidateReceipt, CoreState, DisputeState, GroupRotationInfo, OccupiedCoreAssumption, + BlockNumber, CandidateCommitments, CandidateEvent, CandidateHash, CommittedCandidateReceipt, + CoreState, DisputeState, ExecutorParams, GroupRotationInfo, OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionIndex, SessionInfo, ValidatorId, ValidatorIndex, ValidatorSignature, }; @@ -123,7 +123,7 @@ use sp_std::{collections::btree_map::BTreeMap, prelude::*}; sp_api::decl_runtime_apis! { /// The API for querying the state of parachains on-chain. - #[api_version(2)] + #[api_version(4)] pub trait ParachainHost { /// Get the current validators. fn validators() -> Vec; @@ -213,14 +213,10 @@ sp_api::decl_runtime_apis! { fn validation_code_hash(para_id: ppp::Id, assumption: OccupiedCoreAssumption) -> Option; - /***** STAGING *****/ - /// Returns all onchain disputes. - #[api_version(3)] fn disputes() -> Vec<(SessionIndex, CandidateHash, DisputeState)>; /// Returns execution parameters for the session. - #[api_version(4)] - fn session_executor_params(session_index: SessionIndex) -> Option; + fn session_executor_params(session_index: SessionIndex) -> Option; } } diff --git a/primitives/src/vstaging/executor_params.rs b/primitives/src/v4/executor_params.rs similarity index 100% rename from primitives/src/vstaging/executor_params.rs rename to primitives/src/v4/executor_params.rs diff --git a/primitives/src/v2/metrics.rs b/primitives/src/v4/metrics.rs similarity index 100% rename from primitives/src/v2/metrics.rs rename to primitives/src/v4/metrics.rs diff --git a/primitives/src/v2/mod.rs b/primitives/src/v4/mod.rs similarity index 99% rename from primitives/src/v2/mod.rs rename to primitives/src/v4/mod.rs index 27ad3fb278c5..82d7954b8090 100644 --- a/primitives/src/v2/mod.rs +++ b/primitives/src/v4/mod.rs @@ -143,12 +143,12 @@ pub mod well_known_keys { // // The `StorageValue`, such as `ACTIVE_CONFIG` was obtained by calling: // - // ::ActiveConfig::hashed_key() + // ActiveConfig::::hashed_key() // // The `StorageMap` values require `prefix`, and for example for `hrmp_egress_channel_index`, // it could be obtained like: // - // ::HrmpEgressChannelsIndex::prefix_hash(); + // HrmpEgressChannelsIndex::::prefix_hash(); // /// The current epoch index. @@ -1716,6 +1716,9 @@ pub enum PvfExecTimeoutKind { Approval, } +pub mod executor_params; +pub use executor_params::{ExecutorParam, ExecutorParams, ExecutorParamsHash}; + #[cfg(test)] mod tests { use super::*; diff --git a/primitives/src/v2/signed.rs b/primitives/src/v4/signed.rs similarity index 100% rename from primitives/src/v2/signed.rs rename to primitives/src/v4/signed.rs diff --git a/primitives/src/vstaging/mod.rs b/primitives/src/vstaging/mod.rs index d6428d252149..64671bd48a60 100644 --- a/primitives/src/vstaging/mod.rs +++ b/primitives/src/vstaging/mod.rs @@ -17,6 +17,3 @@ //! Staging Primitives. // Put any primitives used by staging APIs functions here - -pub mod executor_params; -pub use executor_params::{ExecutorParam, ExecutorParams, ExecutorParamsHash}; diff --git a/roadmap/implementers-guide/src/node/utility/pvf-prechecker.md b/roadmap/implementers-guide/src/node/utility/pvf-prechecker.md index 90193ec00e18..3cae12e65f33 100644 --- a/roadmap/implementers-guide/src/node/utility/pvf-prechecker.md +++ b/roadmap/implementers-guide/src/node/utility/pvf-prechecker.md @@ -35,6 +35,12 @@ Rejecting instead of abstaining is better in several ways: Also, if we only abstain, an attacker can specially craft a PVF wasm blob so that it will fail on e.g. 50% of the validators. In that case a supermajority will never be reached and the vote will repeat multiple times, most likely with the same result (since all votes are cleared on a session change). This is avoided by rejecting failed PVFs, and by only requiring 1/3 of validators to reject a PVF to reach a decision. +### Note on Disputes + +Having a pre-checking phase allows us to make certain assumptions later when preparing the PVF for execution. If a runtime passed pre-checking, then we know that the runtime should be valid, and therefore any issue during preparation for execution can be assumed to be a local problem on the current node. + +For this reason, even deterministic preparation errors should not trigger disputes. And since we do not dispute as a result of the pre-checking phase, as stated above, it should be impossible for preparation in general to result in disputes. + [overview]: ../../pvf-prechecking.md [Runtime API]: runtime-api.md [PVF pre-checking runtime API]: ../../runtime-api/pvf-prechecking.md diff --git a/roadmap/implementers-guide/src/pvf-prechecking.md b/roadmap/implementers-guide/src/pvf-prechecking.md index fd2ca12330bd..155d32d52898 100644 --- a/roadmap/implementers-guide/src/pvf-prechecking.md +++ b/roadmap/implementers-guide/src/pvf-prechecking.md @@ -46,7 +46,7 @@ The logic described above is implemented by the [paras] module. On the node-side, there is a PVF pre-checking [subsystem][pvf-prechecker-subsystem] that scans the chain for new PVFs via using [runtime APIs][pvf-runtime-api]. Upon finding a new PVF, the subsystem will initiate a PVF pre-checking request and wait for the result. Whenever the result is obtained, the subsystem will use the [runtime API][pvf-runtime-api] to submit a vote for the PVF. The vote is an unsigned transaction. The vote will be distributed via the gossip similarly to a normal transaction. Eventually a block producer will include the vote into the block where it will be handled by the [runtime][paras]. -## Pre-checking Summary +## Summary Parachains' and parathreads' validation function is described by a wasm module that we refer to as a PVF. @@ -54,7 +54,7 @@ In order to make the PVF usable for candidate validation it has to be registered As part of the registration process, it has to go through pre-checking. Pre-checking is a game of attempting preparation and reporting the results back on-chain. -We define preparation as a process that: validates the consistency of the wasm binary (aka prevalidation) and the compilation of the wasm module into machine code (refered to as artifact). +We define preparation as a process that: validates the consistency of the wasm binary (aka prevalidation) and the compilation of the wasm module into machine code (referred to as an artifact). Besides pre-checking, preparation can also be triggered by execution, since a compiled artifact is needed for the execution. If an artifact already exists, execution will skip preparation. If it does do preparation, execution uses a more lenient timeout than preparation, to avoid the situation where honest validators fail on valid, pre-checked PVFs. diff --git a/runtime/common/src/assigned_slots.rs b/runtime/common/src/assigned_slots.rs index 5186b255168a..96e14820f751 100644 --- a/runtime/common/src/assigned_slots.rs +++ b/runtime/common/src/assigned_slots.rs @@ -78,7 +78,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] pub struct Pallet(_); #[pallet::config] diff --git a/runtime/common/src/auctions.rs b/runtime/common/src/auctions.rs index 41924ff2e514..a75ca812f7b0 100644 --- a/runtime/common/src/auctions.rs +++ b/runtime/common/src/auctions.rs @@ -84,7 +84,6 @@ pub mod pallet { use frame_system::{ensure_root, ensure_signed, pallet_prelude::*}; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] pub struct Pallet(_); /// The module's configuration trait. diff --git a/runtime/common/src/claims.rs b/runtime/common/src/claims.rs index 3c40aab11dc7..82bca143e18b 100644 --- a/runtime/common/src/claims.rs +++ b/runtime/common/src/claims.rs @@ -164,7 +164,6 @@ pub mod pallet { use frame_system::pallet_prelude::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/common/src/crowdloan/mod.rs b/runtime/common/src/crowdloan/mod.rs index e8ef2ee0344f..9f70ac3dd895 100644 --- a/runtime/common/src/crowdloan/mod.rs +++ b/runtime/common/src/crowdloan/mod.rs @@ -186,7 +186,6 @@ pub mod pallet { const STORAGE_VERSION: StorageVersion = StorageVersion::new(1); #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] #[pallet::storage_version(STORAGE_VERSION)] pub struct Pallet(_); diff --git a/runtime/common/src/paras_registrar.rs b/runtime/common/src/paras_registrar.rs index ec65cac94414..2fe16f141683 100644 --- a/runtime/common/src/paras_registrar.rs +++ b/runtime/common/src/paras_registrar.rs @@ -97,7 +97,6 @@ pub mod pallet { use frame_system::pallet_prelude::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/common/src/paras_sudo_wrapper.rs b/runtime/common/src/paras_sudo_wrapper.rs index 5cf6e74e28a6..33b9d18071a3 100644 --- a/runtime/common/src/paras_sudo_wrapper.rs +++ b/runtime/common/src/paras_sudo_wrapper.rs @@ -33,7 +33,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] pub struct Pallet(_); #[pallet::config] diff --git a/runtime/common/src/purchase.rs b/runtime/common/src/purchase.rs index ed4647531ff0..47190c5f2690 100644 --- a/runtime/common/src/purchase.rs +++ b/runtime/common/src/purchase.rs @@ -91,7 +91,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/common/src/slots/mod.rs b/runtime/common/src/slots/mod.rs index 23781cd89338..2c3fd39ae00a 100644 --- a/runtime/common/src/slots/mod.rs +++ b/runtime/common/src/slots/mod.rs @@ -67,7 +67,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/kusama/src/lib.rs b/runtime/kusama/src/lib.rs index fb61273892e1..daa5b7d5f1a7 100644 --- a/runtime/kusama/src/lib.rs +++ b/runtime/kusama/src/lib.rs @@ -23,11 +23,11 @@ use pallet_nis::WithMaximumOf; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CommittedCandidateReceipt, - CoreState, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, - Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, - SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - LOWEST_PUBLIC_ID, + AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, + Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, + OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionInfo, Signature, + ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, LOWEST_PUBLIC_ID, }; use runtime_common::{ auctions, claims, crowdloan, impl_runtime_weights, impls::DealWithFees, paras_registrar, @@ -41,7 +41,7 @@ use runtime_parachains::{ dmp as parachains_dmp, hrmp as parachains_hrmp, inclusion as parachains_inclusion, initializer as parachains_initializer, origin as parachains_origin, paras as parachains_paras, paras_inherent as parachains_paras_inherent, reward_points as parachains_reward_points, - runtime_api_impl::v2 as parachains_runtime_api_impl, scheduler as parachains_scheduler, + runtime_api_impl::v4 as parachains_runtime_api_impl, scheduler as parachains_scheduler, session_info as parachains_session_info, shared as parachains_shared, ump as parachains_ump, }; @@ -125,13 +125,13 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { spec_name: create_runtime_str!("kusama"), impl_name: create_runtime_str!("parity-kusama"), authoring_version: 2, - spec_version: 9370, + spec_version: 9390, impl_version: 0, #[cfg(not(feature = "disable-runtime-api"))] apis: RUNTIME_API_VERSIONS, #[cfg(feature = "disable-runtime-api")] apis: sp_version::create_apis_vec![[]], - transaction_version: 19, + transaction_version: 20, state_version: 0, }; @@ -433,6 +433,7 @@ impl pallet_election_provider_multi_phase::MinerConfig for Runtime { as frame_election_provider_support::ElectionDataProvider >::MaxVotesPerVoter; + type MaxWinners = MaxActiveValidators; // The unsigned submissions have to respect the weight of the submit_unsigned call, thus their // weight estimate function is wired to this call's weight. @@ -1455,17 +1456,22 @@ pub type SignedExtra = ( pallet_transaction_payment::ChargeTransactionPayment, ); -pub struct StakingMigrationV11OldPallet; -impl Get<&'static str> for StakingMigrationV11OldPallet { - fn get() -> &'static str { - "VoterList" +pub struct NominationPoolsMigrationV4OldPallet; +impl Get for NominationPoolsMigrationV4OldPallet { + fn get() -> Perbill { + Perbill::from_percent(10) } } /// All migrations that will run on the next runtime upgrade. /// /// Should be cleared after every release. -pub type Migrations = (); +pub type Migrations = ( + pallet_nomination_pools::migration::v4::MigrateToV4< + Runtime, + NominationPoolsMigrationV4OldPallet, + >, +); /// Unchecked extrinsic type as expected by this runtime. pub type UncheckedExtrinsic = @@ -1672,6 +1678,10 @@ sp_api::impl_runtime_apis! { parachains_runtime_api_impl::session_info::(index) } + fn session_executor_params(session_index: SessionIndex) -> Option { + parachains_runtime_api_impl::session_executor_params::(session_index) + } + fn dmq_contents(recipient: ParaId) -> Vec> { parachains_runtime_api_impl::dmq_contents::(recipient) } @@ -1706,6 +1716,10 @@ sp_api::impl_runtime_apis! { { parachains_runtime_api_impl::validation_code_hash::(para_id, assumption) } + + fn disputes() -> Vec<(SessionIndex, CandidateHash, DisputeState)> { + parachains_runtime_api_impl::get_session_disputes::() + } } impl beefy_primitives::BeefyApi for Runtime { @@ -2256,7 +2270,7 @@ mod multiplier_tests { #[cfg(all(test, feature = "try-runtime"))] mod remote_tests { use super::*; - use frame_try_runtime::{runtime_decl_for_TryRuntime::TryRuntime, UpgradeCheckSelect}; + use frame_try_runtime::{runtime_decl_for_try_runtime::TryRuntime, UpgradeCheckSelect}; use remote_externalities::{ Builder, Mode, OfflineConfig, OnlineConfig, SnapshotConfig, Transport, }; diff --git a/runtime/kusama/src/weights/pallet_nomination_pools.rs b/runtime/kusama/src/weights/pallet_nomination_pools.rs index e1e59fc43693..5bf5914995f1 100644 --- a/runtime/kusama/src/weights/pallet_nomination_pools.rs +++ b/runtime/kusama/src/weights/pallet_nomination_pools.rs @@ -16,22 +16,24 @@ //! Autogenerated weights for `pallet_nomination_pools` //! //! THIS FILE WAS AUTO-GENERATED USING THE SUBSTRATE BENCHMARK CLI VERSION 4.0.0-dev -//! DATE: 2023-02-28, STEPS: `50`, REPEAT: `20`, LOW RANGE: `[]`, HIGH RANGE: `[]` +//! DATE: 2023-03-06, STEPS: `50`, REPEAT: `20`, LOW RANGE: `[]`, HIGH RANGE: `[]` //! WORST CASE MAP SIZE: `1000000` -//! HOSTNAME: `bm5`, CPU: `Intel(R) Core(TM) i7-7700K CPU @ 4.20GHz` +//! HOSTNAME: `bm3`, CPU: `Intel(R) Core(TM) i7-7700K CPU @ 4.20GHz` //! EXECUTION: Some(Wasm), WASM-EXECUTION: Compiled, CHAIN: Some("kusama-dev"), DB CACHE: 1024 // Executed Command: -// ./target/production/polkadot +// target/production/polkadot // benchmark // pallet -// --chain=kusama-dev // --steps=50 // --repeat=20 -// --pallet=pallet_nomination_pools // --extrinsic=* // --execution=wasm // --wasm-execution=compiled +// --heap-pages=4096 +// --json-file=/var/lib/gitlab-runner/builds/zyw4fam_/0/parity/mirrors/polkadot/.git/.artifacts/bench.json +// --pallet=pallet_nomination_pools +// --chain=kusama-dev // --header=./file_header.txt // --output=./runtime/kusama/src/weights/ @@ -50,13 +52,15 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:2 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: NominationPools MaxPoolMembersPerPool (r:1 w:0) @@ -73,20 +77,22 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn join() -> Weight { // Proof Size summary in bytes: - // Measured: `3501` - // Estimated: `38468` - // Minimum execution time: 155_259 nanoseconds. - Weight::from_parts(159_392_000, 0) - .saturating_add(Weight::from_parts(0, 38468)) - .saturating_add(T::DbWeight::get().reads(17)) + // Measured: `3578` + // Estimated: `52915` + // Minimum execution time: 155_563_000 picoseconds. + Weight::from_parts(156_968_000, 0) + .saturating_add(Weight::from_parts(0, 52915)) + .saturating_add(T::DbWeight::get().reads(18)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:3 w:2) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -101,12 +107,12 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn bond_extra_transfer() -> Weight { // Proof Size summary in bytes: - // Measured: `3543` - // Estimated: `39063` - // Minimum execution time: 150_628 nanoseconds. - Weight::from_parts(152_414_000, 0) - .saturating_add(Weight::from_parts(0, 39063)) - .saturating_add(T::DbWeight::get().reads(14)) + // Measured: `3620` + // Estimated: `49550` + // Minimum execution time: 153_086_000 picoseconds. + Weight::from_parts(154_699_000, 0) + .saturating_add(Weight::from_parts(0, 49550)) + .saturating_add(T::DbWeight::get().reads(15)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools ClaimPermissions (r:1 w:0) @@ -114,9 +120,11 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:3 w:3) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -125,50 +133,54 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) /// Storage: Balances Locks (r:1 w:1) /// Proof: Balances Locks (max_values: None, max_size: Some(1299), added: 3774, mode: MaxEncodedLen) - /// Storage: VoterList ListNodes (r:2 w:2) + /// Storage: VoterList ListNodes (r:3 w:3) /// Proof: VoterList ListNodes (max_values: None, max_size: Some(154), added: 2629, mode: MaxEncodedLen) /// Storage: VoterList ListBags (r:2 w:2) /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn bond_extra_other() -> Weight { // Proof Size summary in bytes: - // Measured: `3389` - // Estimated: `38950` - // Minimum execution time: 163_983 nanoseconds. - Weight::from_parts(165_428_000, 0) - .saturating_add(Weight::from_parts(0, 38950)) - .saturating_add(T::DbWeight::get().reads(14)) - .saturating_add(T::DbWeight::get().writes(12)) + // Measured: `3685` + // Estimated: `53056` + // Minimum execution time: 172_493_000 picoseconds. + Weight::from_parts(173_786_000, 0) + .saturating_add(Weight::from_parts(0, 53056)) + .saturating_add(T::DbWeight::get().reads(16)) + .saturating_add(T::DbWeight::get().writes(13)) } /// Storage: NominationPools ClaimPermissions (r:1 w:0) /// Proof: NominationPools ClaimPermissions (max_values: None, max_size: Some(41), added: 2516, mode: MaxEncodedLen) /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:1 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) fn claim_payout() -> Weight { // Proof Size summary in bytes: - // Measured: `1254` - // Estimated: `13485` - // Minimum execution time: 59_520 nanoseconds. - Weight::from_parts(60_666_000, 0) - .saturating_add(Weight::from_parts(0, 13485)) - .saturating_add(T::DbWeight::get().reads(5)) + // Measured: `1331` + // Estimated: `20012` + // Minimum execution time: 60_276_000 picoseconds. + Weight::from_parts(60_972_000, 0) + .saturating_add(Weight::from_parts(0, 20012)) + .saturating_add(T::DbWeight::get().reads(6)) .saturating_add(T::DbWeight::get().writes(4)) } /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:2 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking CurrentEra (r:1 w:0) @@ -189,16 +201,16 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: NominationPools CounterForSubPoolsStorage (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn unbond() -> Weight { // Proof Size summary in bytes: - // Measured: `3779` - // Estimated: `44930` - // Minimum execution time: 154_659 nanoseconds. - Weight::from_parts(156_090_000, 0) - .saturating_add(Weight::from_parts(0, 44930)) - .saturating_add(T::DbWeight::get().reads(18)) + // Measured: `3856` + // Estimated: `60367` + // Minimum execution time: 156_830_000 picoseconds. + Weight::from_parts(158_198_000, 0) + .saturating_add(Weight::from_parts(0, 60367)) + .saturating_add(T::DbWeight::get().reads(19)) .saturating_add(T::DbWeight::get().writes(13)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) @@ -210,13 +222,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `s` is `[0, 100]`. fn pool_withdraw_unbonded(s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `1700` - // Estimated: `13025` - // Minimum execution time: 52_836 nanoseconds. - Weight::from_parts(54_086_254, 0) - .saturating_add(Weight::from_parts(0, 13025)) - // Standard Error: 753 - .saturating_add(Weight::from_parts(8_103, 0).saturating_mul(s.into())) + // Measured: `1704` + // Estimated: `18031` + // Minimum execution time: 53_206_000 picoseconds. + Weight::from_parts(54_391_455, 0) + .saturating_add(Weight::from_parts(0, 18031)) + // Standard Error: 2_654 + .saturating_add(Weight::from_parts(10_172, 0).saturating_mul(s.into())) .saturating_add(T::DbWeight::get().reads(5)) .saturating_add(T::DbWeight::get().writes(2)) } @@ -225,7 +237,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Staking CurrentEra (r:1 w:0) /// Proof: Staking CurrentEra (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools SubPoolsStorage (r:1 w:1) /// Proof: NominationPools SubPoolsStorage (max_values: None, max_size: Some(1197), added: 3672, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -241,15 +253,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools ClaimPermissions (r:0 w:1) /// Proof: NominationPools ClaimPermissions (max_values: None, max_size: Some(41), added: 2516, mode: MaxEncodedLen) /// The range of component `s` is `[0, 100]`. - fn withdraw_unbonded_update(s: u32, ) -> Weight { + fn withdraw_unbonded_update(_s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `2224` - // Estimated: `22991` - // Minimum execution time: 102_419 nanoseconds. - Weight::from_parts(104_332_875, 0) - .saturating_add(Weight::from_parts(0, 22991)) - // Standard Error: 1_933 - .saturating_add(Weight::from_parts(15_901, 0).saturating_mul(s.into())) + // Measured: `2228` + // Estimated: `31957` + // Minimum execution time: 102_755_000 picoseconds. + Weight::from_parts(107_719_238, 0) + .saturating_add(Weight::from_parts(0, 31957)) .saturating_add(T::DbWeight::get().reads(9)) .saturating_add(T::DbWeight::get().writes(8)) } @@ -258,7 +268,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Staking CurrentEra (r:1 w:0) /// Proof: Staking CurrentEra (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools SubPoolsStorage (r:1 w:1) /// Proof: NominationPools SubPoolsStorage (max_values: None, max_size: Some(1197), added: 3672, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:1) @@ -282,7 +292,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools CounterForReversePoolIdLookup (r:1 w:1) /// Proof: NominationPools CounterForReversePoolIdLookup (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: NominationPools CounterForRewardPools (r:1 w:1) /// Proof: NominationPools CounterForRewardPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools CounterForSubPoolsStorage (r:1 w:1) @@ -296,13 +306,15 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools ClaimPermissions (r:0 w:1) /// Proof: NominationPools ClaimPermissions (max_values: None, max_size: Some(41), added: 2516, mode: MaxEncodedLen) /// The range of component `s` is `[0, 100]`. - fn withdraw_unbonded_kill(_s: u32, ) -> Weight { + fn withdraw_unbonded_kill(s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `2614` - // Estimated: `46287` - // Minimum execution time: 162_308 nanoseconds. - Weight::from_parts(165_347_893, 0) - .saturating_add(Weight::from_parts(0, 46287)) + // Measured: `2618` + // Estimated: `65189` + // Minimum execution time: 161_317_000 picoseconds. + Weight::from_parts(164_663_249, 0) + .saturating_add(Weight::from_parts(0, 65189)) + // Standard Error: 25_821 + .saturating_add(Weight::from_parts(58_147, 0).saturating_mul(s.into())) .saturating_add(T::DbWeight::get().reads(20)) .saturating_add(T::DbWeight::get().writes(18)) } @@ -337,7 +349,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Balances Locks (r:1 w:1) /// Proof: Balances Locks (max_values: None, max_size: Some(1299), added: 3774, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: NominationPools CounterForRewardPools (r:1 w:1) /// Proof: NominationPools CounterForRewardPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools ReversePoolIdLookup (r:1 w:1) @@ -345,21 +357,21 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools CounterForReversePoolIdLookup (r:1 w:1) /// Proof: NominationPools CounterForReversePoolIdLookup (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Payee (r:0 w:1) /// Proof: Staking Payee (max_values: None, max_size: Some(73), added: 2548, mode: MaxEncodedLen) fn create() -> Weight { // Proof Size summary in bytes: // Measured: `1254` - // Estimated: `32002` - // Minimum execution time: 144_295 nanoseconds. - Weight::from_parts(145_923_000, 0) - .saturating_add(Weight::from_parts(0, 32002)) + // Estimated: `51890` + // Minimum execution time: 141_784_000 picoseconds. + Weight::from_parts(142_683_000, 0) + .saturating_add(Weight::from_parts(0, 51890)) .saturating_add(T::DbWeight::get().reads(21)) .saturating_add(T::DbWeight::get().writes(15)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) @@ -385,36 +397,36 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `n` is `[1, 24]`. fn nominate(n: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `1834` - // Estimated: `22254 + n * (2520 ±0)` - // Minimum execution time: 66_817 nanoseconds. - Weight::from_parts(67_646_764, 0) - .saturating_add(Weight::from_parts(0, 22254)) - // Standard Error: 12_768 - .saturating_add(Weight::from_parts(1_325_294, 0).saturating_mul(n.into())) + // Measured: `1838` + // Estimated: `34190 + n * (2520 ±0)` + // Minimum execution time: 65_561_000 picoseconds. + Weight::from_parts(68_033_566, 0) + .saturating_add(Weight::from_parts(0, 34190)) + // Standard Error: 41_874 + .saturating_add(Weight::from_parts(1_249_632, 0).saturating_mul(n.into())) .saturating_add(T::DbWeight::get().reads(12)) .saturating_add(T::DbWeight::get().reads((1_u64).saturating_mul(n.into()))) .saturating_add(T::DbWeight::get().writes(5)) .saturating_add(Weight::from_parts(0, 2520).saturating_mul(n.into())) } /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) fn set_state() -> Weight { // Proof Size summary in bytes: - // Measured: `1423` - // Estimated: `8752` - // Minimum execution time: 35_795 nanoseconds. - Weight::from_parts(36_450_000, 0) - .saturating_add(Weight::from_parts(0, 8752)) + // Measured: `1427` + // Estimated: `11778` + // Minimum execution time: 36_008_000 picoseconds. + Weight::from_parts(36_665_000, 0) + .saturating_add(Weight::from_parts(0, 11778)) .saturating_add(T::DbWeight::get().reads(3)) .saturating_add(T::DbWeight::get().writes(1)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools Metadata (r:1 w:1) /// Proof: NominationPools Metadata (max_values: None, max_size: Some(270), added: 2745, mode: MaxEncodedLen) /// Storage: NominationPools CounterForMetadata (r:1 w:1) @@ -422,13 +434,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `n` is `[1, 256]`. fn set_metadata(n: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `559` - // Estimated: `5883` - // Minimum execution time: 14_066 nanoseconds. - Weight::from_parts(14_523_246, 0) - .saturating_add(Weight::from_parts(0, 5883)) - // Standard Error: 114 - .saturating_add(Weight::from_parts(1_692, 0).saturating_mul(n.into())) + // Measured: `563` + // Estimated: `8909` + // Minimum execution time: 14_479_000 picoseconds. + Weight::from_parts(15_744_317, 0) + .saturating_add(Weight::from_parts(0, 8909)) + // Standard Error: 1_213 + .saturating_add(Weight::from_parts(2_309, 0).saturating_mul(n.into())) .saturating_add(T::DbWeight::get().reads(3)) .saturating_add(T::DbWeight::get().writes(2)) } @@ -440,31 +452,33 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: NominationPools MaxPoolMembersPerPool (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools MinCreateBond (r:0 w:1) /// Proof: NominationPools MinCreateBond (max_values: Some(1), max_size: Some(16), added: 511, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:0 w:1) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools MaxPools (r:0 w:1) /// Proof: NominationPools MaxPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn set_configs() -> Weight { // Proof Size summary in bytes: // Measured: `0` // Estimated: `0` - // Minimum execution time: 5_088 nanoseconds. - Weight::from_parts(5_337_000, 0) + // Minimum execution time: 6_288_000 picoseconds. + Weight::from_parts(6_550_000, 0) .saturating_add(Weight::from_parts(0, 0)) - .saturating_add(T::DbWeight::get().writes(5)) + .saturating_add(T::DbWeight::get().writes(6)) } /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) fn update_roles() -> Weight { // Proof Size summary in bytes: - // Measured: `559` - // Estimated: `2639` - // Minimum execution time: 19_312 nanoseconds. - Weight::from_parts(19_483_000, 0) - .saturating_add(Weight::from_parts(0, 2639)) + // Measured: `563` + // Estimated: `3685` + // Minimum execution time: 20_297_000 picoseconds. + Weight::from_parts(20_698_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) .saturating_add(T::DbWeight::get().reads(1)) .saturating_add(T::DbWeight::get().writes(1)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) @@ -483,14 +497,56 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList CounterForListNodes (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn chill() -> Weight { // Proof Size summary in bytes: - // Measured: `2061` - // Estimated: `20745` - // Minimum execution time: 65_721 nanoseconds. - Weight::from_parts(66_514_000, 0) - .saturating_add(Weight::from_parts(0, 20745)) + // Measured: `2065` + // Estimated: `29711` + // Minimum execution time: 64_288_000 picoseconds. + Weight::from_parts(65_089_000, 0) + .saturating_add(Weight::from_parts(0, 29711)) .saturating_add(T::DbWeight::get().reads(9)) .saturating_add(T::DbWeight::get().writes(5)) } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + /// Storage: NominationPools RewardPools (r:1 w:1) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) + /// Storage: System Account (r:1 w:0) + /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) + fn set_commission() -> Weight { + // Proof Size summary in bytes: + // Measured: `866` + // Estimated: `12324` + // Minimum execution time: 32_180_000 picoseconds. + Weight::from_parts(32_637_000, 0) + .saturating_add(Weight::from_parts(0, 12324)) + .saturating_add(T::DbWeight::get().reads(4)) + .saturating_add(T::DbWeight::get().writes(2)) + } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + fn set_commission_max() -> Weight { + // Proof Size summary in bytes: + // Measured: `603` + // Estimated: `3685` + // Minimum execution time: 18_814_000 picoseconds. + Weight::from_parts(19_221_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) + .saturating_add(T::DbWeight::get().reads(1)) + .saturating_add(T::DbWeight::get().writes(1)) + } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + fn set_commission_change_rate() -> Weight { + // Proof Size summary in bytes: + // Measured: `563` + // Estimated: `3685` + // Minimum execution time: 19_641_000 picoseconds. + Weight::from_parts(29_789_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) + .saturating_add(T::DbWeight::get().reads(1)) + .saturating_add(T::DbWeight::get().writes(1)) + } /// Storage: NominationPools PoolMembers (r:1 w:0) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools ClaimPermissions (r:1 w:1) @@ -498,11 +554,29 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo fn set_claim_permission() -> Weight { // Proof Size summary in bytes: // Measured: `542` - // Estimated: `5708` - // Minimum execution time: 14_637 nanoseconds. - Weight::from_parts(15_053_000, 0) - .saturating_add(Weight::from_parts(0, 5708)) + // Estimated: `7688` + // Minimum execution time: 14_819_000 picoseconds. + Weight::from_parts(21_194_000, 0) + .saturating_add(Weight::from_parts(0, 7688)) .saturating_add(T::DbWeight::get().reads(2)) .saturating_add(T::DbWeight::get().writes(1)) } + /// Storage: NominationPools BondedPools (r:1 w:0) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + /// Storage: NominationPools RewardPools (r:1 w:1) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) + /// Storage: System Account (r:1 w:1) + /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) + fn claim_commission() -> Weight { + // Proof Size summary in bytes: + // Measured: `1096` + // Estimated: `12324` + // Minimum execution time: 47_609_000 picoseconds. + Weight::from_parts(48_694_000, 0) + .saturating_add(Weight::from_parts(0, 12324)) + .saturating_add(T::DbWeight::get().reads(4)) + .saturating_add(T::DbWeight::get().writes(2)) + } } diff --git a/runtime/parachains/src/configuration.rs b/runtime/parachains/src/configuration.rs index 9bc061f071a9..14926c726b48 100644 --- a/runtime/parachains/src/configuration.rs +++ b/runtime/parachains/src/configuration.rs @@ -473,7 +473,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::storage_version(migration::STORAGE_VERSION)] #[pallet::without_storage_info] pub struct Pallet(_); @@ -1155,7 +1154,7 @@ pub mod pallet { ))] pub fn set_bypass_consistency_check(origin: OriginFor, new: bool) -> DispatchResult { ensure_root(origin)?; - ::BypassConsistencyCheck::put(new); + BypassConsistencyCheck::::put(new); Ok(()) } } @@ -1200,7 +1199,7 @@ impl Pallet { session_index: &SessionIndex, ) -> SessionChangeOutcome { let pending_configs = >::get(); - let prev_config = ::ActiveConfig::get(); + let prev_config = ActiveConfig::::get(); // No pending configuration changes, so we're done. if pending_configs.is_empty() { @@ -1223,7 +1222,7 @@ impl Pallet { let new_config = past_and_present.pop().map(|(_, config)| config); if let Some(ref new_config) = new_config { // Apply the new configuration. - ::ActiveConfig::put(new_config); + ActiveConfig::::put(new_config); } >::put(future); @@ -1240,7 +1239,7 @@ impl Pallet { /// only when enabling parachains runtime pallets for the first time on a chain which has /// been running without them. pub fn force_set_active_config(config: HostConfiguration) { - ::ActiveConfig::set(config); + ActiveConfig::::set(config); } /// This function should be used to update members of the configuration. @@ -1302,7 +1301,7 @@ impl Pallet { updater(&mut base_config); let new_config = base_config; - if ::BypassConsistencyCheck::get() { + if BypassConsistencyCheck::::get() { // This will emit a warning each configuration update if the consistency check is // bypassed. This is an attempt to make sure the bypass is not accidentally left on. log::warn!( diff --git a/runtime/parachains/src/configuration/migration.rs b/runtime/parachains/src/configuration/migration.rs index abef3c0ccc11..7b2092cfc2c1 100644 --- a/runtime/parachains/src/configuration/migration.rs +++ b/runtime/parachains/src/configuration/migration.rs @@ -16,7 +16,7 @@ //! A module that is responsible for migration of storage. -use crate::configuration::{self, Config, Pallet, Store, MAX_POV_SIZE}; +use crate::configuration::{self, ActiveConfig, Config, Pallet, MAX_POV_SIZE}; use frame_support::{pallet_prelude::*, traits::StorageVersion, weights::Weight}; use frame_system::pallet_prelude::BlockNumberFor; @@ -230,7 +230,7 @@ minimum_validation_upgrade_delay : pre.minimum_validation_upgrade_delay, } }; - if let Err(_) = as Store>::ActiveConfig::translate(|pre| pre.map(translate)) { + if let Err(_) = ActiveConfig::::translate(|pre| pre.map(translate)) { // `Err` is returned when the pre-migration type cannot be deserialized. This // cannot happen if the migration runs correctly, i.e. against the expected version. // diff --git a/runtime/parachains/src/configuration/tests.rs b/runtime/parachains/src/configuration/tests.rs index 52ce3929dadb..2d89aebc19d3 100644 --- a/runtime/parachains/src/configuration/tests.rs +++ b/runtime/parachains/src/configuration/tests.rs @@ -72,18 +72,18 @@ fn config_changes_after_2_session_boundary() { // Verify that the current configuration has not changed and that there is a scheduled // change for the SESSION_DELAY sessions in advance. assert_eq!(Configuration::config(), old_config); - assert_eq!(::PendingConfigs::get(), vec![(2, config.clone())]); + assert_eq!(PendingConfigs::::get(), vec![(2, config.clone())]); on_new_session(1); // One session has passed, we should be still waiting for the pending configuration. assert_eq!(Configuration::config(), old_config); - assert_eq!(::PendingConfigs::get(), vec![(2, config.clone())]); + assert_eq!(PendingConfigs::::get(), vec![(2, config.clone())]); on_new_session(2); assert_eq!(Configuration::config(), config); - assert_eq!(::PendingConfigs::get(), vec![]); + assert_eq!(PendingConfigs::::get(), vec![]); }) } @@ -99,17 +99,17 @@ fn consecutive_changes_within_one_session() { assert_ok!(Configuration::set_validation_upgrade_delay(RuntimeOrigin::root(), 100)); assert_ok!(Configuration::set_validation_upgrade_cooldown(RuntimeOrigin::root(), 100)); assert_eq!(Configuration::config(), old_config); - assert_eq!(::PendingConfigs::get(), vec![(2, config.clone())]); + assert_eq!(PendingConfigs::::get(), vec![(2, config.clone())]); on_new_session(1); assert_eq!(Configuration::config(), old_config); - assert_eq!(::PendingConfigs::get(), vec![(2, config.clone())]); + assert_eq!(PendingConfigs::::get(), vec![(2, config.clone())]); on_new_session(2); assert_eq!(Configuration::config(), config); - assert_eq!(::PendingConfigs::get(), vec![]); + assert_eq!(PendingConfigs::::get(), vec![]); }); } @@ -127,10 +127,7 @@ fn pending_next_session_but_we_upgrade_once_more() { assert_ok!(Configuration::set_validation_upgrade_delay(RuntimeOrigin::root(), 100)); assert_eq!(Configuration::config(), initial_config); - assert_eq!( - ::PendingConfigs::get(), - vec![(2, intermediate_config.clone())] - ); + assert_eq!(PendingConfigs::::get(), vec![(2, intermediate_config.clone())]); on_new_session(1); @@ -141,22 +138,19 @@ fn pending_next_session_but_we_upgrade_once_more() { // This should result in yet another configiguration change scheduled. assert_eq!(Configuration::config(), initial_config); assert_eq!( - ::PendingConfigs::get(), + PendingConfigs::::get(), vec![(2, intermediate_config.clone()), (3, final_config.clone())] ); on_new_session(2); assert_eq!(Configuration::config(), intermediate_config); - assert_eq!( - ::PendingConfigs::get(), - vec![(3, final_config.clone())] - ); + assert_eq!(PendingConfigs::::get(), vec![(3, final_config.clone())]); on_new_session(3); assert_eq!(Configuration::config(), final_config); - assert_eq!(::PendingConfigs::get(), vec![]); + assert_eq!(PendingConfigs::::get(), vec![]); }); } @@ -175,10 +169,7 @@ fn scheduled_session_config_update_while_next_session_pending() { assert_ok!(Configuration::set_validation_upgrade_delay(RuntimeOrigin::root(), 100)); assert_eq!(Configuration::config(), initial_config); - assert_eq!( - ::PendingConfigs::get(), - vec![(2, intermediate_config.clone())] - ); + assert_eq!(PendingConfigs::::get(), vec![(2, intermediate_config.clone())]); on_new_session(1); @@ -190,22 +181,19 @@ fn scheduled_session_config_update_while_next_session_pending() { // This should result in yet another configiguration change scheduled. assert_eq!(Configuration::config(), initial_config); assert_eq!( - ::PendingConfigs::get(), + PendingConfigs::::get(), vec![(2, intermediate_config.clone()), (3, final_config.clone())] ); on_new_session(2); assert_eq!(Configuration::config(), intermediate_config); - assert_eq!( - ::PendingConfigs::get(), - vec![(3, final_config.clone())] - ); + assert_eq!(PendingConfigs::::get(), vec![(3, final_config.clone())]); on_new_session(3); assert_eq!(Configuration::config(), final_config); - assert_eq!(::PendingConfigs::get(), vec![]); + assert_eq!(PendingConfigs::::get(), vec![]); }); } @@ -240,7 +228,7 @@ fn invariants() { Error::::InvalidNewValue ); - ::ActiveConfig::put(HostConfiguration { + ActiveConfig::::put(HostConfiguration { chain_availability_period: 10, thread_availability_period: 8, minimum_validation_upgrade_delay: 11, @@ -514,10 +502,7 @@ fn setting_pending_config_members() { Configuration::set_pvf_voting_ttl(RuntimeOrigin::root(), new_config.pvf_voting_ttl) .unwrap(); - assert_eq!( - ::PendingConfigs::get(), - vec![(shared::SESSION_DELAY, new_config)], - ); + assert_eq!(PendingConfigs::::get(), vec![(shared::SESSION_DELAY, new_config)],); }) } @@ -539,7 +524,7 @@ fn verify_externally_accessible() { let ground_truth = HostConfiguration::default(); // Make sure that the configuration is stored in the storage. - ::ActiveConfig::put(ground_truth.clone()); + ActiveConfig::::put(ground_truth.clone()); // Extract the active config via the well known key. let raw_active_config = sp_io::storage::get(well_known_keys::ACTIVE_CONFIG) diff --git a/runtime/parachains/src/dmp.rs b/runtime/parachains/src/dmp.rs index 855ae1bd85b6..03a767eb428f 100644 --- a/runtime/parachains/src/dmp.rs +++ b/runtime/parachains/src/dmp.rs @@ -75,7 +75,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); @@ -135,8 +134,8 @@ impl Pallet { /// Remove all relevant storage items for an outgoing parachain. fn clean_dmp_after_outgoing(outgoing_para: &ParaId) { - ::DownwardMessageQueues::remove(outgoing_para); - ::DownwardMessageQueueHeads::remove(outgoing_para); + DownwardMessageQueues::::remove(outgoing_para); + DownwardMessageQueueHeads::::remove(outgoing_para); } /// Determine whether enqueuing a downward message to a specific recipient para would result @@ -152,9 +151,7 @@ impl Pallet { return Err(QueueDownwardMessageError::ExceedsMaxMessageSize) } - if ::DownwardMessageQueues::decode_len(para).unwrap_or(0) > - MAX_MESSAGE_QUEUE_SIZE - { + if DownwardMessageQueues::::decode_len(para).unwrap_or(0) > MAX_MESSAGE_QUEUE_SIZE { return Err(QueueDownwardMessageError::ExceedsMaxMessageSize) } @@ -179,9 +176,7 @@ impl Pallet { return Err(QueueDownwardMessageError::ExceedsMaxMessageSize) } - if ::DownwardMessageQueues::decode_len(para).unwrap_or(0) > - MAX_MESSAGE_QUEUE_SIZE - { + if DownwardMessageQueues::::decode_len(para).unwrap_or(0) > MAX_MESSAGE_QUEUE_SIZE { return Err(QueueDownwardMessageError::ExceedsMaxMessageSize) } @@ -189,13 +184,13 @@ impl Pallet { InboundDownwardMessage { msg, sent_at: >::block_number() }; // obtain the new link in the MQC and update the head. - ::DownwardMessageQueueHeads::mutate(para, |head| { + DownwardMessageQueueHeads::::mutate(para, |head| { let new_head = BlakeTwo256::hash_of(&(*head, inbound.sent_at, T::Hashing::hash_of(&inbound.msg))); *head = new_head; }); - ::DownwardMessageQueues::mutate(para, |v| { + DownwardMessageQueues::::mutate(para, |v| { v.push(inbound); }); @@ -224,7 +219,7 @@ impl Pallet { /// Prunes the specified number of messages from the downward message queue of the given para. pub(crate) fn prune_dmq(para: ParaId, processed_downward_messages: u32) -> Weight { - ::DownwardMessageQueues::mutate(para, |q| { + DownwardMessageQueues::::mutate(para, |q| { let processed_downward_messages = processed_downward_messages as usize; if processed_downward_messages > q.len() { // reaching this branch is unexpected due to the constraint established by @@ -241,14 +236,14 @@ impl Pallet { /// associated with it. #[cfg(test)] fn dmq_mqc_head(para: ParaId) -> Hash { - ::DownwardMessageQueueHeads::get(¶) + DownwardMessageQueueHeads::::get(¶) } /// Returns the number of pending downward messages addressed to the given para. /// /// Returns 0 if the para doesn't have an associated downward message queue. pub(crate) fn dmq_length(para: ParaId) -> u32 { - ::DownwardMessageQueues::decode_len(¶) + DownwardMessageQueues::::decode_len(¶) .unwrap_or(0) .saturated_into::() } @@ -257,6 +252,6 @@ impl Pallet { /// /// The most recent messages are the latest in the vector. pub(crate) fn dmq_contents(recipient: ParaId) -> Vec> { - ::DownwardMessageQueues::get(&recipient) + DownwardMessageQueues::::get(&recipient) } } diff --git a/runtime/parachains/src/dmp/tests.rs b/runtime/parachains/src/dmp/tests.rs index b19c2db0914a..a3d9b6e3ac85 100644 --- a/runtime/parachains/src/dmp/tests.rs +++ b/runtime/parachains/src/dmp/tests.rs @@ -15,7 +15,7 @@ // along with Polkadot. If not, see . use super::*; -use crate::mock::{new_test_ext, Configuration, Dmp, MockGenesisConfig, Paras, System}; +use crate::mock::{new_test_ext, Configuration, Dmp, MockGenesisConfig, Paras, System, Test}; use hex_literal::hex; use parity_scale_codec::Encode; use primitives::BlockNumber; @@ -73,9 +73,9 @@ fn clean_dmp_works() { let outgoing_paras = vec![a, b]; Dmp::initializer_on_new_session(¬ification, &outgoing_paras); - assert!(::DownwardMessageQueues::get(&a).is_empty()); - assert!(::DownwardMessageQueues::get(&b).is_empty()); - assert!(!::DownwardMessageQueues::get(&c).is_empty()); + assert!(DownwardMessageQueues::::get(&a).is_empty()); + assert!(DownwardMessageQueues::::get(&b).is_empty()); + assert!(!DownwardMessageQueues::::get(&c).is_empty()); }); } diff --git a/runtime/parachains/src/hrmp.rs b/runtime/parachains/src/hrmp.rs index 396ba1f5f73b..fe87f85db757 100644 --- a/runtime/parachains/src/hrmp.rs +++ b/runtime/parachains/src/hrmp.rs @@ -235,7 +235,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); @@ -584,8 +583,8 @@ pub mod pallet { ) -> DispatchResult { let origin = ensure_parachain(::RuntimeOrigin::from(origin))?; ensure!( - ::HrmpOpenChannelRequestsList::decode_len().unwrap_or_default() - as u32 <= open_requests, + HrmpOpenChannelRequestsList::::decode_len().unwrap_or_default() as u32 <= + open_requests, Error::::WrongWitness ); Self::cancel_open_request(origin, channel_id.clone())?; @@ -685,10 +684,10 @@ impl Pallet { // we need a few extra bits of data to weigh this -- all of this is read internally // anyways, so no overhead. - let ingress_count = ::HrmpIngressChannelsIndex::decode_len(outgoing_para) - .unwrap_or_default() as u32; - let egress_count = ::HrmpEgressChannelsIndex::decode_len(outgoing_para) - .unwrap_or_default() as u32; + let ingress_count = + HrmpIngressChannelsIndex::::decode_len(outgoing_para).unwrap_or_default() as u32; + let egress_count = + HrmpEgressChannelsIndex::::decode_len(outgoing_para).unwrap_or_default() as u32; w = w.saturating_add(::WeightInfo::force_clean_hrmp( ingress_count, egress_count, @@ -709,16 +708,16 @@ impl Pallet { // // Both the open channel request list and outgoing list are expected to be small enough. // In the most common case there will be only single outgoing para. - let open_channel_reqs = ::HrmpOpenChannelRequestsList::get(); + let open_channel_reqs = HrmpOpenChannelRequestsList::::get(); let (go, stay): (Vec, Vec) = open_channel_reqs .into_iter() .partition(|req_id| outgoing.iter().any(|id| req_id.is_participant(*id))); - ::HrmpOpenChannelRequestsList::put(stay); + HrmpOpenChannelRequestsList::::put(stay); // Then iterate over all open requests to be removed, pull them out of the set and perform // the refunds if applicable. for req_id in go { - let req_data = match ::HrmpOpenChannelRequests::take(&req_id) { + let req_data = match HrmpOpenChannelRequests::::take(&req_id) { Some(req_data) => req_data, None => { // Can't normally happen but no need to panic. @@ -755,13 +754,13 @@ impl Pallet { /// Remove all storage entries associated with the given para. fn clean_hrmp_after_outgoing(outgoing_para: &ParaId) { - ::HrmpOpenChannelRequestCount::remove(outgoing_para); - ::HrmpAcceptedChannelRequestCount::remove(outgoing_para); + HrmpOpenChannelRequestCount::::remove(outgoing_para); + HrmpAcceptedChannelRequestCount::::remove(outgoing_para); - let ingress = ::HrmpIngressChannelsIndex::take(outgoing_para) + let ingress = HrmpIngressChannelsIndex::::take(outgoing_para) .into_iter() .map(|sender| HrmpChannelId { sender, recipient: *outgoing_para }); - let egress = ::HrmpEgressChannelsIndex::take(outgoing_para) + let egress = HrmpEgressChannelsIndex::::take(outgoing_para) .into_iter() .map(|recipient| HrmpChannelId { sender: *outgoing_para, recipient }); let mut to_close = ingress.chain(egress).collect::>(); @@ -778,7 +777,7 @@ impl Pallet { /// - prune the stale requests /// - enact the confirmed requests fn process_hrmp_open_channel_requests(config: &HostConfiguration) { - let mut open_req_channels = ::HrmpOpenChannelRequestsList::get(); + let mut open_req_channels = HrmpOpenChannelRequestsList::::get(); if open_req_channels.is_empty() { return } @@ -794,7 +793,7 @@ impl Pallet { idx -= 1; let channel_id = open_req_channels[idx].clone(); - let request = ::HrmpOpenChannelRequests::get(&channel_id).expect( + let request = HrmpOpenChannelRequests::::get(&channel_id).expect( "can't be `None` due to the invariant that the list contains the same items as the set; qed", ); @@ -802,7 +801,7 @@ impl Pallet { if >::is_valid_para(channel_id.sender) && >::is_valid_para(channel_id.recipient) { - ::HrmpChannels::insert( + HrmpChannels::::insert( &channel_id, HrmpChannel { sender_deposit: request.sender_deposit, @@ -816,12 +815,12 @@ impl Pallet { }, ); - ::HrmpIngressChannelsIndex::mutate(&channel_id.recipient, |v| { + HrmpIngressChannelsIndex::::mutate(&channel_id.recipient, |v| { if let Err(i) = v.binary_search(&channel_id.sender) { v.insert(i, channel_id.sender); } }); - ::HrmpEgressChannelsIndex::mutate(&channel_id.sender, |v| { + HrmpEgressChannelsIndex::::mutate(&channel_id.sender, |v| { if let Err(i) = v.binary_search(&channel_id.recipient) { v.insert(i, channel_id.recipient); } @@ -832,18 +831,18 @@ impl Pallet { Self::decrease_accepted_channel_request_count(channel_id.recipient); let _ = open_req_channels.swap_remove(idx); - ::HrmpOpenChannelRequests::remove(&channel_id); + HrmpOpenChannelRequests::::remove(&channel_id); } } - ::HrmpOpenChannelRequestsList::put(open_req_channels); + HrmpOpenChannelRequestsList::::put(open_req_channels); } /// Iterate over all close channel requests unconditionally closing the channels. fn process_hrmp_close_channel_requests() { - let close_reqs = ::HrmpCloseChannelRequestsList::take(); + let close_reqs = HrmpCloseChannelRequestsList::::take(); for condemned_ch_id in close_reqs { - ::HrmpCloseChannelRequests::remove(&condemned_ch_id); + HrmpCloseChannelRequests::::remove(&condemned_ch_id); Self::close_hrmp_channel(&condemned_ch_id); } } @@ -856,7 +855,7 @@ impl Pallet { /// effect (i.e. it won't return the deposits twice). fn close_hrmp_channel(channel_id: &HrmpChannelId) { if let Some(HrmpChannel { sender_deposit, recipient_deposit, .. }) = - ::HrmpChannels::take(channel_id) + HrmpChannels::::take(channel_id) { T::Currency::unreserve( &channel_id.sender.into_account_truncating(), @@ -868,14 +867,14 @@ impl Pallet { ); } - ::HrmpChannelContents::remove(channel_id); + HrmpChannelContents::::remove(channel_id); - ::HrmpEgressChannelsIndex::mutate(&channel_id.sender, |v| { + HrmpEgressChannelsIndex::::mutate(&channel_id.sender, |v| { if let Ok(i) = v.binary_search(&channel_id.recipient) { v.remove(i); } }); - ::HrmpIngressChannelsIndex::mutate(&channel_id.recipient, |v| { + HrmpIngressChannelsIndex::::mutate(&channel_id.recipient, |v| { if let Ok(i) = v.binary_search(&channel_id.sender) { v.remove(i); } @@ -895,7 +894,7 @@ impl Pallet { // // (b) However, a parachain cannot read into "the future", therefore the watermark should // not be greater than the relay-chain context block which the parablock refers to. - if let Some(last_watermark) = ::HrmpWatermarks::get(&recipient) { + if let Some(last_watermark) = HrmpWatermarks::::get(&recipient) { if new_hrmp_watermark <= last_watermark { return Err(HrmpWatermarkAcceptanceErr::AdvancementRule { new_watermark: new_hrmp_watermark, @@ -917,7 +916,7 @@ impl Pallet { if new_hrmp_watermark == relay_chain_parent_number { Ok(()) } else { - let digest = ::HrmpChannelDigests::get(&recipient); + let digest = HrmpChannelDigests::::get(&recipient); if !digest .binary_search_by_key(&new_hrmp_watermark, |(block_no, _)| *block_no) .is_ok() @@ -958,7 +957,7 @@ impl Pallet { let channel_id = HrmpChannelId { sender, recipient: out_msg.recipient }; - let channel = match ::HrmpChannels::get(&channel_id) { + let channel = match HrmpChannels::::get(&channel_id) { Some(channel) => channel, None => return Err(OutboundHrmpAcceptanceErr::NoSuchChannel { channel_id, idx }), }; @@ -999,7 +998,7 @@ impl Pallet { // sift through the incoming messages digest to collect the paras that sent at least one // message to this parachain between the old and new watermarks. - let senders = ::HrmpChannelDigests::mutate(&recipient, |digest| { + let senders = HrmpChannelDigests::::mutate(&recipient, |digest| { let mut senders = BTreeSet::new(); let mut leftover = Vec::with_capacity(digest.len()); for (block_no, paras_sent_msg) in mem::replace(digest, Vec::new()) { @@ -1022,7 +1021,7 @@ impl Pallet { // and what is the total byte size of them. let (mut pruned_cnt, mut pruned_size) = (0, 0); - let contents = ::HrmpChannelContents::get(&channel_id); + let contents = HrmpChannelContents::::get(&channel_id); let mut leftover = Vec::with_capacity(contents.len()); for msg in contents { if msg.sent_at <= new_hrmp_watermark { @@ -1033,13 +1032,13 @@ impl Pallet { } } if !leftover.is_empty() { - ::HrmpChannelContents::insert(&channel_id, leftover); + HrmpChannelContents::::insert(&channel_id, leftover); } else { - ::HrmpChannelContents::remove(&channel_id); + HrmpChannelContents::::remove(&channel_id); } // update the channel metadata. - ::HrmpChannels::mutate(&channel_id, |channel| { + HrmpChannels::::mutate(&channel_id, |channel| { if let Some(ref mut channel) = channel { channel.msg_count -= pruned_cnt as u32; channel.total_size -= pruned_size as u32; @@ -1049,7 +1048,7 @@ impl Pallet { weight += T::DbWeight::get().reads_writes(2, 2); } - ::HrmpWatermarks::insert(&recipient, new_hrmp_watermark); + HrmpWatermarks::::insert(&recipient, new_hrmp_watermark); weight += T::DbWeight::get().reads_writes(0, 1); weight @@ -1065,7 +1064,7 @@ impl Pallet { for out_msg in out_hrmp_msgs { let channel_id = HrmpChannelId { sender, recipient: out_msg.recipient }; - let mut channel = match ::HrmpChannels::get(&channel_id) { + let mut channel = match HrmpChannels::::get(&channel_id) { Some(channel) => channel, None => { // apparently, that since acceptance of this candidate the recipient was @@ -1089,8 +1088,8 @@ impl Pallet { )); channel.mqc_head = Some(new_head); - ::HrmpChannels::insert(&channel_id, channel); - ::HrmpChannelContents::append(&channel_id, inbound); + HrmpChannels::::insert(&channel_id, channel); + HrmpChannelContents::::append(&channel_id, inbound); // The digests are sorted in ascending by block number order. Assuming absence of // contextual execution, there are only two possible scenarios here: @@ -1104,8 +1103,7 @@ impl Pallet { // // Note that having the latest entry greater than the current block number is a logical // error. - let mut recipient_digest = - ::HrmpChannelDigests::get(&channel_id.recipient); + let mut recipient_digest = HrmpChannelDigests::::get(&channel_id.recipient); if let Some(cur_block_digest) = recipient_digest .last_mut() .filter(|(block_no, _)| *block_no == now) @@ -1115,7 +1113,7 @@ impl Pallet { } else { recipient_digest.push((now, vec![sender])); } - ::HrmpChannelDigests::insert(&channel_id.recipient, recipient_digest); + HrmpChannelDigests::::insert(&channel_id.recipient, recipient_digest); weight += T::DbWeight::get().reads_writes(2, 2); } @@ -1154,17 +1152,16 @@ impl Pallet { let channel_id = HrmpChannelId { sender: origin, recipient }; ensure!( - ::HrmpOpenChannelRequests::get(&channel_id).is_none(), + HrmpOpenChannelRequests::::get(&channel_id).is_none(), Error::::OpenHrmpChannelAlreadyRequested, ); ensure!( - ::HrmpChannels::get(&channel_id).is_none(), + HrmpChannels::::get(&channel_id).is_none(), Error::::OpenHrmpChannelAlreadyExists, ); - let egress_cnt = - ::HrmpEgressChannelsIndex::decode_len(&origin).unwrap_or(0) as u32; - let open_req_cnt = ::HrmpOpenChannelRequestCount::get(&origin); + let egress_cnt = HrmpEgressChannelsIndex::::decode_len(&origin).unwrap_or(0) as u32; + let open_req_cnt = HrmpOpenChannelRequestCount::::get(&origin); let channel_num_limit = if >::is_parathread(origin) { config.hrmp_max_parathread_outbound_channels } else { @@ -1182,8 +1179,8 @@ impl Pallet { // mutating storage directly now -- shall not bail henceforth. - ::HrmpOpenChannelRequestCount::insert(&origin, open_req_cnt + 1); - ::HrmpOpenChannelRequests::insert( + HrmpOpenChannelRequestCount::::insert(&origin, open_req_cnt + 1); + HrmpOpenChannelRequests::::insert( &channel_id, HrmpOpenChannelRequest { confirmed: false, @@ -1194,7 +1191,7 @@ impl Pallet { max_total_size: config.hrmp_channel_max_total_size, }, ); - ::HrmpOpenChannelRequestsList::append(channel_id); + HrmpOpenChannelRequestsList::::append(channel_id); let notification_bytes = { use parity_scale_codec::Encode as _; @@ -1228,7 +1225,7 @@ impl Pallet { /// intended for calling directly from other pallets rather than dispatched. pub fn accept_open_channel(origin: ParaId, sender: ParaId) -> DispatchResult { let channel_id = HrmpChannelId { sender, recipient: origin }; - let mut channel_req = ::HrmpOpenChannelRequests::get(&channel_id) + let mut channel_req = HrmpOpenChannelRequests::::get(&channel_id) .ok_or(Error::::AcceptHrmpChannelDoesntExist)?; ensure!(!channel_req.confirmed, Error::::AcceptHrmpChannelAlreadyConfirmed); @@ -1240,9 +1237,8 @@ impl Pallet { } else { config.hrmp_max_parachain_inbound_channels }; - let ingress_cnt = - ::HrmpIngressChannelsIndex::decode_len(&origin).unwrap_or(0) as u32; - let accepted_cnt = ::HrmpAcceptedChannelRequestCount::get(&origin); + let ingress_cnt = HrmpIngressChannelsIndex::::decode_len(&origin).unwrap_or(0) as u32; + let accepted_cnt = HrmpAcceptedChannelRequestCount::::get(&origin); ensure!( ingress_cnt + accepted_cnt < channel_num_limit, Error::::AcceptHrmpChannelLimitExceeded, @@ -1256,8 +1252,8 @@ impl Pallet { // persist the updated open channel request and then increment the number of accepted // channels. channel_req.confirmed = true; - ::HrmpOpenChannelRequests::insert(&channel_id, channel_req); - ::HrmpAcceptedChannelRequestCount::insert(&origin, accepted_cnt + 1); + HrmpOpenChannelRequests::::insert(&channel_id, channel_req); + HrmpAcceptedChannelRequestCount::::insert(&origin, accepted_cnt + 1); let notification_bytes = { use parity_scale_codec::Encode as _; @@ -1284,13 +1280,13 @@ impl Pallet { // check if the origin is allowed to close the channel. ensure!(channel_id.is_participant(origin), Error::::CancelHrmpOpenChannelUnauthorized); - let open_channel_req = ::HrmpOpenChannelRequests::get(&channel_id) + let open_channel_req = HrmpOpenChannelRequests::::get(&channel_id) .ok_or(Error::::OpenHrmpChannelDoesntExist)?; ensure!(!open_channel_req.confirmed, Error::::OpenHrmpChannelAlreadyConfirmed); // Remove the request by the channel id and sync the accompanying list with the set. - ::HrmpOpenChannelRequests::remove(&channel_id); - ::HrmpOpenChannelRequestsList::mutate(|open_req_channels| { + HrmpOpenChannelRequests::::remove(&channel_id); + HrmpOpenChannelRequestsList::::mutate(|open_req_channels| { if let Some(pos) = open_req_channels.iter().position(|x| x == &channel_id) { open_req_channels.swap_remove(pos); } @@ -1316,18 +1312,18 @@ impl Pallet { // check if the channel requested to close does exist. ensure!( - ::HrmpChannels::get(&channel_id).is_some(), + HrmpChannels::::get(&channel_id).is_some(), Error::::CloseHrmpChannelDoesntExist, ); // check that there is no outstanding close request for this channel ensure!( - ::HrmpCloseChannelRequests::get(&channel_id).is_none(), + HrmpCloseChannelRequests::::get(&channel_id).is_none(), Error::::CloseHrmpChannelAlreadyUnderway, ); - ::HrmpCloseChannelRequests::insert(&channel_id, ()); - ::HrmpCloseChannelRequestsList::append(channel_id.clone()); + HrmpCloseChannelRequests::::insert(&channel_id, ()); + HrmpCloseChannelRequestsList::::append(channel_id.clone()); let config = >::config(); let notification_bytes = { @@ -1363,13 +1359,12 @@ impl Pallet { /// multiple entries with the same sender. #[cfg(test)] fn hrmp_mqc_heads(recipient: ParaId) -> Vec<(ParaId, Hash)> { - let sender_set = ::HrmpIngressChannelsIndex::get(&recipient); + let sender_set = HrmpIngressChannelsIndex::::get(&recipient); // The ingress channels vector is sorted, thus `mqc_heads` is sorted as well. let mut mqc_heads = Vec::with_capacity(sender_set.len()); for sender in sender_set { - let channel_metadata = - ::HrmpChannels::get(&HrmpChannelId { sender, recipient }); + let channel_metadata = HrmpChannels::::get(&HrmpChannelId { sender, recipient }); let mqc_head = channel_metadata .and_then(|metadata| metadata.mqc_head) .unwrap_or(Hash::default()); @@ -1384,12 +1379,12 @@ impl Pallet { pub(crate) fn inbound_hrmp_channels_contents( recipient: ParaId, ) -> BTreeMap>> { - let sender_set = ::HrmpIngressChannelsIndex::get(&recipient); + let sender_set = HrmpIngressChannelsIndex::::get(&recipient); let mut inbound_hrmp_channels_contents = BTreeMap::new(); for sender in sender_set { let channel_contents = - ::HrmpChannelContents::get(&HrmpChannelId { sender, recipient }); + HrmpChannelContents::::get(&HrmpChannelId { sender, recipient }); inbound_hrmp_channels_contents.insert(sender, channel_contents); } @@ -1401,7 +1396,7 @@ impl Pallet { /// Decreases the open channel request count for the given sender. If the value reaches zero /// it is removed completely. fn decrease_open_channel_request_count(sender: ParaId) { - ::HrmpOpenChannelRequestCount::mutate_exists(&sender, |opt_rc| { + HrmpOpenChannelRequestCount::::mutate_exists(&sender, |opt_rc| { *opt_rc = opt_rc.and_then(|rc| match rc.saturating_sub(1) { 0 => None, n => Some(n), @@ -1412,7 +1407,7 @@ impl Pallet { /// Decreases the accepted channel request count for the given sender. If the value reaches /// zero it is removed completely. fn decrease_accepted_channel_request_count(recipient: ParaId) { - ::HrmpAcceptedChannelRequestCount::mutate_exists(&recipient, |opt_rc| { + HrmpAcceptedChannelRequestCount::::mutate_exists(&recipient, |opt_rc| { *opt_rc = opt_rc.and_then(|rc| match rc.saturating_sub(1) { 0 => None, n => Some(n), @@ -1438,12 +1433,8 @@ impl Pallet { }; assert_eq!( - ::HrmpOpenChannelRequests::iter() - .map(|(k, _)| k) - .collect::>(), - ::HrmpOpenChannelRequestsList::get() - .into_iter() - .collect::>(), + HrmpOpenChannelRequests::::iter().map(|(k, _)| k).collect::>(), + HrmpOpenChannelRequestsList::::get().into_iter().collect::>(), ); // verify that the set of keys in `HrmpOpenChannelRequestCount` corresponds to the set @@ -1451,17 +1442,15 @@ impl Pallet { // // having ensured that, we can go ahead and go over all counts and verify that they match. assert_eq!( - ::HrmpOpenChannelRequestCount::iter() + HrmpOpenChannelRequestCount::::iter() .map(|(k, _)| k) .collect::>(), - ::HrmpOpenChannelRequests::iter() + HrmpOpenChannelRequests::::iter() .map(|(k, _)| k.sender) .collect::>(), ); - for (open_channel_initiator, expected_num) in - ::HrmpOpenChannelRequestCount::iter() - { - let actual_num = ::HrmpOpenChannelRequests::iter() + for (open_channel_initiator, expected_num) in HrmpOpenChannelRequestCount::::iter() { + let actual_num = HrmpOpenChannelRequests::::iter() .filter(|(ch, _)| ch.sender == open_channel_initiator) .count() as u32; assert_eq!(expected_num, actual_num); @@ -1470,43 +1459,37 @@ impl Pallet { // The same as above, but for accepted channel request count. Note that we are interested // only in confirmed open requests. assert_eq!( - ::HrmpAcceptedChannelRequestCount::iter() + HrmpAcceptedChannelRequestCount::::iter() .map(|(k, _)| k) .collect::>(), - ::HrmpOpenChannelRequests::iter() + HrmpOpenChannelRequests::::iter() .filter(|(_, v)| v.confirmed) .map(|(k, _)| k.recipient) .collect::>(), ); - for (channel_recipient, expected_num) in - ::HrmpAcceptedChannelRequestCount::iter() - { - let actual_num = ::HrmpOpenChannelRequests::iter() + for (channel_recipient, expected_num) in HrmpAcceptedChannelRequestCount::::iter() { + let actual_num = HrmpOpenChannelRequests::::iter() .filter(|(ch, v)| ch.recipient == channel_recipient && v.confirmed) .count() as u32; assert_eq!(expected_num, actual_num); } assert_eq!( - ::HrmpCloseChannelRequests::iter() - .map(|(k, _)| k) - .collect::>(), - ::HrmpCloseChannelRequestsList::get() - .into_iter() - .collect::>(), + HrmpCloseChannelRequests::::iter().map(|(k, _)| k).collect::>(), + HrmpCloseChannelRequestsList::::get().into_iter().collect::>(), ); // A HRMP watermark can be None for an onboarded parachain. However, an offboarded parachain // cannot have an HRMP watermark: it should've been cleanup. assert_contains_only_onboarded( - ::HrmpWatermarks::iter().map(|(k, _)| k).collect::>(), + HrmpWatermarks::::iter().map(|(k, _)| k).collect::>(), "HRMP watermarks should contain only onboarded paras", ); // An entry in `HrmpChannels` indicates that the channel is open. Only open channels can // have contents. - for (non_empty_channel, contents) in ::HrmpChannelContents::iter() { - assert!(::HrmpChannels::contains_key(&non_empty_channel)); + for (non_empty_channel, contents) in HrmpChannelContents::::iter() { + assert!(HrmpChannels::::contains_key(&non_empty_channel)); // pedantic check: there should be no empty vectors in storage, those should be modeled // by a removed kv pair. @@ -1516,7 +1499,7 @@ impl Pallet { // Senders and recipients must be onboarded. Otherwise, all channels associated with them // are removed. assert_contains_only_onboarded( - ::HrmpChannels::iter() + HrmpChannels::::iter() .flat_map(|(k, _)| vec![k.sender, k.recipient]) .collect::>(), "senders and recipients in all channels should be onboarded", @@ -1541,30 +1524,30 @@ impl Pallet { // (b, z) (b, z) // // and then that we compare that to the channel list in the `HrmpChannels`. - let channel_set_derived_from_ingress = ::HrmpIngressChannelsIndex::iter() + let channel_set_derived_from_ingress = HrmpIngressChannelsIndex::::iter() .flat_map(|(p, v)| v.into_iter().map(|i| (i, p)).collect::>()) .collect::>(); - let channel_set_derived_from_egress = ::HrmpEgressChannelsIndex::iter() + let channel_set_derived_from_egress = HrmpEgressChannelsIndex::::iter() .flat_map(|(p, v)| v.into_iter().map(|e| (p, e)).collect::>()) .collect::>(); - let channel_set_ground_truth = ::HrmpChannels::iter() + let channel_set_ground_truth = HrmpChannels::::iter() .map(|(k, _)| (k.sender, k.recipient)) .collect::>(); assert_eq!(channel_set_derived_from_ingress, channel_set_derived_from_egress); assert_eq!(channel_set_derived_from_egress, channel_set_ground_truth); - ::HrmpIngressChannelsIndex::iter() + HrmpIngressChannelsIndex::::iter() .map(|(_, v)| v) .for_each(|v| assert_is_sorted(&v, "HrmpIngressChannelsIndex")); - ::HrmpEgressChannelsIndex::iter() + HrmpEgressChannelsIndex::::iter() .map(|(_, v)| v) .for_each(|v| assert_is_sorted(&v, "HrmpIngressChannelsIndex")); assert_contains_only_onboarded( - ::HrmpChannelDigests::iter().map(|(k, _)| k).collect::>(), + HrmpChannelDigests::::iter().map(|(k, _)| k).collect::>(), "HRMP channel digests should contain only onboarded paras", ); - for (_digest_for_para, digest) in ::HrmpChannelDigests::iter() { + for (_digest_for_para, digest) in HrmpChannelDigests::::iter() { // Assert that items are in **strictly** ascending order. The strictness also implies // there are no duplicates. assert!(digest.windows(2).all(|xs| xs[0].0 < xs[1].0)); diff --git a/runtime/parachains/src/hrmp/tests.rs b/runtime/parachains/src/hrmp/tests.rs index 3c9fedaa9dcb..33737126dbf5 100644 --- a/runtime/parachains/src/hrmp/tests.rs +++ b/runtime/parachains/src/hrmp/tests.rs @@ -150,7 +150,7 @@ fn deregister_parachain(id: ParaId) { } fn channel_exists(sender: ParaId, recipient: ParaId) -> bool { - ::HrmpChannels::get(&HrmpChannelId { sender, recipient }).is_some() + HrmpChannels::::get(&HrmpChannelId { sender, recipient }).is_some() } #[test] diff --git a/runtime/parachains/src/inclusion/mod.rs b/runtime/parachains/src/inclusion/mod.rs index da63abe3a433..d73dac903128 100644 --- a/runtime/parachains/src/inclusion/mod.rs +++ b/runtime/parachains/src/inclusion/mod.rs @@ -184,7 +184,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/parachains/src/initializer.rs b/runtime/parachains/src/initializer.rs index 60b95cf0db9c..8f59b687d97f 100644 --- a/runtime/parachains/src/initializer.rs +++ b/runtime/parachains/src/initializer.rs @@ -99,7 +99,6 @@ pub mod pallet { use frame_system::pallet_prelude::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/parachains/src/initializer/tests.rs b/runtime/parachains/src/initializer/tests.rs index c9522330a4cd..9b5b38446e54 100644 --- a/runtime/parachains/src/initializer/tests.rs +++ b/runtime/parachains/src/initializer/tests.rs @@ -17,6 +17,7 @@ use super::*; use crate::mock::{ new_test_ext, Configuration, Dmp, Initializer, MockGenesisConfig, Paras, SessionInfo, System, + Test, }; use primitives::{HeadData, Id as ParaId}; use test_helpers::dummy_validation_code; @@ -32,7 +33,7 @@ fn session_0_is_instantly_applied() { new_test_ext(Default::default()).execute_with(|| { Initializer::on_new_session(false, 0, Vec::new().into_iter(), Some(Vec::new().into_iter())); - let v = ::BufferedSessionChanges::get(); + let v = BufferedSessionChanges::::get(); assert!(v.is_empty()); assert_eq!(SessionInfo::earliest_stored_session(), 0); @@ -48,7 +49,7 @@ fn session_change_before_initialize_is_still_buffered_after() { let now = System::block_number(); Initializer::on_initialize(now); - let v = ::BufferedSessionChanges::get(); + let v = BufferedSessionChanges::::get(); assert_eq!(v.len(), 1); }); } @@ -61,7 +62,7 @@ fn session_change_applied_on_finalize() { Initializer::on_finalize(1); - assert!(::BufferedSessionChanges::get().is_empty()); + assert!(BufferedSessionChanges::::get().is_empty()); }); } @@ -70,7 +71,7 @@ fn sets_flag_on_initialize() { new_test_ext(Default::default()).execute_with(|| { Initializer::on_initialize(1); - assert!(::HasInitialized::get().is_some()); + assert!(HasInitialized::::get().is_some()); }) } @@ -80,7 +81,7 @@ fn clears_flag_on_finalize() { Initializer::on_initialize(1); Initializer::on_finalize(1); - assert!(::HasInitialized::get().is_none()); + assert!(HasInitialized::::get().is_none()); }) } diff --git a/runtime/parachains/src/origin.rs b/runtime/parachains/src/origin.rs index aa8096982569..cbb662c6eef3 100644 --- a/runtime/parachains/src/origin.rs +++ b/runtime/parachains/src/origin.rs @@ -46,7 +46,6 @@ pub mod pallet { use frame_support::pallet_prelude::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] pub struct Pallet(_); #[pallet::config] diff --git a/runtime/parachains/src/paras/benchmarking.rs b/runtime/parachains/src/paras/benchmarking.rs index e1ecd98ab4d0..0d961c94bfff 100644 --- a/runtime/parachains/src/paras/benchmarking.rs +++ b/runtime/parachains/src/paras/benchmarking.rs @@ -48,7 +48,7 @@ fn generate_disordered_pruning() { needs_pruning.push((id, block_number)); } - as Store>::PastCodePruning::put(needs_pruning); + PastCodePruning::::put(needs_pruning); } pub(crate) fn generate_disordered_upgrades() { @@ -62,8 +62,8 @@ pub(crate) fn generate_disordered_upgrades() { cooldowns.push((id, block_number)); } - as Store>::UpcomingUpgrades::put(upgrades); - as Store>::UpgradeCooldowns::put(cooldowns); + UpcomingUpgrades::::put(upgrades); + UpgradeCooldowns::::put(cooldowns); } fn generate_disordered_actions_queue() { @@ -75,7 +75,7 @@ fn generate_disordered_actions_queue() { queue.push(id); } - as Store>::ActionsQueue::mutate(next_session, |v| { + ActionsQueue::::mutate(next_session, |v| { *v = queue; }); } @@ -85,7 +85,7 @@ benchmarks! { let c in 1 .. MAX_CODE_SIZE; let new_code = ValidationCode(vec![0; c as usize]); let para_id = ParaId::from(c as u32); - as Store>::CurrentCodeHash::insert(¶_id, new_code.hash()); + CurrentCodeHash::::insert(¶_id, new_code.hash()); generate_disordered_pruning::(); }: _(RawOrigin::Root, para_id, new_code) verify { @@ -114,7 +114,7 @@ benchmarks! { let para_id = ParaId::from(1000); let new_head = HeadData(vec![0; s as usize]); let old_code_hash = ValidationCode(vec![0]).hash(); - as Store>::CurrentCodeHash::insert(¶_id, old_code_hash); + CurrentCodeHash::::insert(¶_id, old_code_hash); // schedule an expired code upgrade for this `para_id` so that force_note_new_head would use // the worst possible code path let expired = frame_system::Pallet::::block_number().saturating_sub(One::one()); diff --git a/runtime/parachains/src/paras/mod.rs b/runtime/parachains/src/paras/mod.rs index 7103b05152d1..420ed3e6409e 100644 --- a/runtime/parachains/src/paras/mod.rs +++ b/runtime/parachains/src/paras/mod.rs @@ -533,7 +533,6 @@ pub mod pallet { }; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); @@ -799,10 +798,10 @@ pub mod pallet { new_code: ValidationCode, ) -> DispatchResult { ensure_root(origin)?; - let maybe_prior_code_hash = ::CurrentCodeHash::get(¶); + let maybe_prior_code_hash = CurrentCodeHash::::get(¶); let new_code_hash = new_code.hash(); Self::increase_code_ref(&new_code_hash, &new_code); - ::CurrentCodeHash::insert(¶, new_code_hash); + CurrentCodeHash::::insert(¶, new_code_hash); let now = frame_system::Pallet::::block_number(); if let Some(prior_code_hash) = maybe_prior_code_hash { @@ -901,7 +900,7 @@ pub mod pallet { ensure_root(origin)?; let code_hash = validation_code.hash(); - if let Some(vote) = ::PvfActiveVoteMap::get(&code_hash) { + if let Some(vote) = PvfActiveVoteMap::::get(&code_hash) { // Remove the existing vote. PvfActiveVoteMap::::remove(&code_hash); PvfActiveVoteList::::mutate(|l| { @@ -921,7 +920,7 @@ pub mod pallet { return Ok(()) } - if ::CodeByHash::contains_key(&code_hash) { + if CodeByHash::::contains_key(&code_hash) { // There is no vote, but the code exists. Nothing to do here. return Ok(()) } @@ -931,7 +930,7 @@ pub mod pallet { // // NOTE That we do not use `increase_code_ref` here, because the code is not yet used // by any parachain. - ::CodeByHash::insert(code_hash, &validation_code); + CodeByHash::::insert(code_hash, &validation_code); Ok(()) } @@ -948,8 +947,8 @@ pub mod pallet { validation_code_hash: ValidationCodeHash, ) -> DispatchResult { ensure_root(origin)?; - if ::CodeByHashRefs::get(&validation_code_hash) == 0 { - ::CodeByHash::remove(&validation_code_hash); + if CodeByHashRefs::::get(&validation_code_hash) == 0 { + CodeByHash::::remove(&validation_code_hash); } Ok(()) } @@ -1151,7 +1150,7 @@ impl Pallet { /// Set the current head of a parachain. pub(crate) fn set_current_head(para: ParaId, new_head: HeadData) { - ::Heads::insert(¶, new_head); + Heads::::insert(¶, new_head); Self::deposit_event(Event::CurrentHeadUpdated(para)); } @@ -1212,7 +1211,7 @@ impl Pallet { None | Some(ParaLifecycle::Parathread) | Some(ParaLifecycle::Parachain) => { /* Nothing to do... */ }, Some(ParaLifecycle::Onboarding) => { - if let Some(genesis_data) = ::UpcomingParasGenesis::take(¶) { + if let Some(genesis_data) = UpcomingParasGenesis::::take(¶) { Self::initialize_para_now(&mut parachains, para, &genesis_data); } }, @@ -1231,17 +1230,17 @@ impl Pallet { Some(ParaLifecycle::OffboardingParathread) => { parachains.remove(para); - ::Heads::remove(¶); - ::FutureCodeUpgrades::remove(¶); - ::UpgradeGoAheadSignal::remove(¶); - ::UpgradeRestrictionSignal::remove(¶); + Heads::::remove(¶); + FutureCodeUpgrades::::remove(¶); + UpgradeGoAheadSignal::::remove(¶); + UpgradeRestrictionSignal::::remove(¶); ParaLifecycles::::remove(¶); - let removed_future_code_hash = ::FutureCodeHash::take(¶); + let removed_future_code_hash = FutureCodeHash::::take(¶); if let Some(removed_future_code_hash) = removed_future_code_hash { Self::decrease_code_ref(&removed_future_code_hash); } - let removed_code_hash = ::CurrentCodeHash::take(¶); + let removed_code_hash = CurrentCodeHash::::take(¶); if let Some(removed_code_hash) = removed_code_hash { Self::note_past_code(para, now, now, removed_code_hash); } @@ -1258,13 +1257,13 @@ impl Pallet { // // NOTE both of those iterates over the list and the outgoing. We do not expect either // of these to be large. Thus should be fine. - ::UpcomingUpgrades::mutate(|upcoming_upgrades| { + UpcomingUpgrades::::mutate(|upcoming_upgrades| { *upcoming_upgrades = mem::take(upcoming_upgrades) .into_iter() .filter(|&(ref para, _)| !outgoing.contains(para)) .collect(); }); - ::UpgradeCooldowns::mutate(|upgrade_cooldowns| { + UpgradeCooldowns::::mutate(|upgrade_cooldowns| { *upgrade_cooldowns = mem::take(upgrade_cooldowns) .into_iter() .filter(|&(ref para, _)| !outgoing.contains(para)) @@ -1290,15 +1289,15 @@ impl Pallet { now: T::BlockNumber, old_code_hash: ValidationCodeHash, ) -> Weight { - ::PastCodeMeta::mutate(&id, |past_meta| { + PastCodeMeta::::mutate(&id, |past_meta| { past_meta.note_replacement(at, now); }); - ::PastCodeHash::insert(&(id, at), old_code_hash); + PastCodeHash::::insert(&(id, at), old_code_hash); // Schedule pruning for this past-code to be removed as soon as it // exits the slashing window. - ::PastCodePruning::mutate(|pruning| { + PastCodePruning::::mutate(|pruning| { let insert_idx = pruning.binary_search_by_key(&now, |&(_, b)| b).unwrap_or_else(|idx| idx); pruning.insert(insert_idx, (id, now)); @@ -1320,8 +1319,8 @@ impl Pallet { // The height of any changes we no longer should keep around. let pruning_height = now - (code_retention_period + One::one()); - let pruning_tasks_done = ::PastCodePruning::mutate( - |pruning_tasks: &mut Vec<(_, T::BlockNumber)>| { + let pruning_tasks_done = + PastCodePruning::::mutate(|pruning_tasks: &mut Vec<(_, T::BlockNumber)>| { let (pruning_tasks_done, pruning_tasks_to_do) = { // find all past code that has just exited the pruning window. let up_to_idx = @@ -1330,10 +1329,10 @@ impl Pallet { }; for (para_id, _) in pruning_tasks_to_do { - let full_deactivate = ::PastCodeMeta::mutate(¶_id, |meta| { + let full_deactivate = PastCodeMeta::::mutate(¶_id, |meta| { for pruned_repl_at in meta.prune_up_to(pruning_height) { let removed_code_hash = - ::PastCodeHash::take(&(para_id, pruned_repl_at)); + PastCodeHash::::take(&(para_id, pruned_repl_at)); if let Some(removed_code_hash) = removed_code_hash { Self::decrease_code_ref(&removed_code_hash); @@ -1352,13 +1351,12 @@ impl Pallet { // This parachain has been removed and now the vestigial code // has been removed from the state. clean up meta as well. if full_deactivate { - ::PastCodeMeta::remove(¶_id); + PastCodeMeta::::remove(¶_id); } } pruning_tasks_done as u64 - }, - ); + }); // 1 read for the meta for each pruning task, 1 read for the config // 2 writes: updating the meta and pruning the code @@ -1373,11 +1371,11 @@ impl Pallet { fn process_scheduled_upgrade_changes(now: T::BlockNumber) -> Weight { // account weight for `UpcomingUpgrades::mutate`. let mut weight = T::DbWeight::get().reads_writes(1, 1); - let upgrades_signaled = ::UpcomingUpgrades::mutate( + let upgrades_signaled = UpcomingUpgrades::::mutate( |upcoming_upgrades: &mut Vec<(ParaId, T::BlockNumber)>| { let num = upcoming_upgrades.iter().take_while(|&(_, at)| at <= &now).count(); for (para, _) in upcoming_upgrades.drain(..num) { - ::UpgradeGoAheadSignal::insert(¶, UpgradeGoAhead::GoAhead); + UpgradeGoAheadSignal::::insert(¶, UpgradeGoAhead::GoAhead); } num }, @@ -1386,10 +1384,8 @@ impl Pallet { // account weight for `UpgradeCooldowns::get`. weight += T::DbWeight::get().reads(1); - let cooldowns_expired = ::UpgradeCooldowns::get() - .iter() - .take_while(|&(_, at)| at <= &now) - .count(); + let cooldowns_expired = + UpgradeCooldowns::::get().iter().take_while(|&(_, at)| at <= &now).count(); // reserve weight for `initializer_finalize`: // - 1 read and 1 write for `UpgradeCooldowns::mutate`. @@ -1404,13 +1400,11 @@ impl Pallet { /// /// See `process_scheduled_upgrade_changes` for more details. fn process_scheduled_upgrade_cooldowns(now: T::BlockNumber) { - ::UpgradeCooldowns::mutate( - |upgrade_cooldowns: &mut Vec<(ParaId, T::BlockNumber)>| { - for &(para, _) in upgrade_cooldowns.iter().take_while(|&(_, at)| at <= &now) { - ::UpgradeRestrictionSignal::remove(¶); - } - }, - ); + UpgradeCooldowns::::mutate(|upgrade_cooldowns: &mut Vec<(ParaId, T::BlockNumber)>| { + for &(para, _) in upgrade_cooldowns.iter().take_while(|&(_, at)| at <= &now) { + UpgradeRestrictionSignal::::remove(¶); + } + }); } /// Goes over all PVF votes in progress, reinitializes ballots, increments ages and prunes the @@ -1539,7 +1533,7 @@ impl Pallet { weight += T::DbWeight::get().reads_writes(1, 4); FutureCodeUpgrades::::insert(&id, expected_at); - ::UpcomingUpgrades::mutate(|upcoming_upgrades| { + UpcomingUpgrades::::mutate(|upcoming_upgrades| { let insert_idx = upcoming_upgrades .binary_search_by_key(&expected_at, |&(_, b)| b) .unwrap_or_else(|idx| idx); @@ -1653,7 +1647,7 @@ impl Pallet { mem::replace(&mut genesis_data.validation_code, ValidationCode(Vec::new())); UpcomingParasGenesis::::insert(&id, genesis_data); let validation_code_hash = validation_code.hash(); - ::CurrentCodeHash::insert(&id, validation_code_hash); + CurrentCodeHash::::insert(&id, validation_code_hash); let cfg = configuration::Pallet::::config(); Self::kick_off_pvf_check( @@ -1814,7 +1808,7 @@ impl Pallet { weight += T::DbWeight::get().reads_writes(1, 1); let next_possible_upgrade_at = relay_parent_number + cfg.validation_upgrade_cooldown; - ::UpgradeCooldowns::mutate(|upgrade_cooldowns| { + UpgradeCooldowns::::mutate(|upgrade_cooldowns| { let insert_idx = upgrade_cooldowns .binary_search_by_key(&next_possible_upgrade_at, |&(_, b)| b) .unwrap_or_else(|idx| idx); @@ -1926,10 +1920,10 @@ impl Pallet { ) -> Weight { Heads::::insert(&id, new_head); - if let Some(expected_at) = ::FutureCodeUpgrades::get(&id) { + if let Some(expected_at) = FutureCodeUpgrades::::get(&id) { if expected_at <= execution_context { - ::FutureCodeUpgrades::remove(&id); - ::UpgradeGoAheadSignal::remove(&id); + FutureCodeUpgrades::::remove(&id); + UpgradeGoAheadSignal::::remove(&id); // Both should always be `Some` in this case, since a code upgrade is scheduled. let new_code_hash = if let Some(new_code_hash) = FutureCodeHash::::take(&id) { @@ -2048,10 +2042,10 @@ impl Pallet { /// Returns the weight consumed. fn increase_code_ref(code_hash: &ValidationCodeHash, code: &ValidationCode) -> Weight { let mut weight = T::DbWeight::get().reads_writes(1, 1); - ::CodeByHashRefs::mutate(code_hash, |refs| { + CodeByHashRefs::::mutate(code_hash, |refs| { if *refs == 0 { weight += T::DbWeight::get().writes(1); - ::CodeByHash::insert(code_hash, code); + CodeByHash::::insert(code_hash, code); } *refs += 1; }); @@ -2064,18 +2058,18 @@ impl Pallet { /// Returns the weight consumed. fn decrease_code_ref(code_hash: &ValidationCodeHash) -> Weight { let mut weight = T::DbWeight::get().reads(1); - let refs = ::CodeByHashRefs::get(code_hash); + let refs = CodeByHashRefs::::get(code_hash); if refs == 0 { log::error!(target: LOG_TARGET, "Code refs is already zero for {:?}", code_hash); return weight } if refs <= 1 { weight += T::DbWeight::get().writes(2); - ::CodeByHash::remove(code_hash); - ::CodeByHashRefs::remove(code_hash); + CodeByHash::::remove(code_hash); + CodeByHashRefs::::remove(code_hash); } else { weight += T::DbWeight::get().writes(1); - ::CodeByHashRefs::insert(code_hash, refs - 1); + CodeByHashRefs::::insert(code_hash, refs - 1); } weight } diff --git a/runtime/parachains/src/paras/tests.rs b/runtime/parachains/src/paras/tests.rs index 98233887b61f..89acfc9bc9b9 100644 --- a/runtime/parachains/src/paras/tests.rs +++ b/runtime/parachains/src/paras/tests.rs @@ -102,13 +102,13 @@ fn upgrade_at( } fn check_code_is_stored(validation_code: &ValidationCode) { - assert!(::CodeByHashRefs::get(validation_code.hash()) != 0); - assert!(::CodeByHash::contains_key(validation_code.hash())); + assert!(CodeByHashRefs::::get(validation_code.hash()) != 0); + assert!(CodeByHash::::contains_key(validation_code.hash())); } fn check_code_is_not_stored(validation_code: &ValidationCode) { - assert!(!::CodeByHashRefs::contains_key(validation_code.hash())); - assert!(!::CodeByHash::contains_key(validation_code.hash())); + assert!(!CodeByHashRefs::::contains_key(validation_code.hash())); + assert!(!CodeByHash::::contains_key(validation_code.hash())); } /// An utility for checking that certain events were deposited. @@ -276,32 +276,26 @@ fn para_past_code_pruning_in_initialize() { let validation_code = ValidationCode(vec![4, 5, 6]); Paras::increase_code_ref(&validation_code.hash(), &validation_code); - ::PastCodeHash::insert(&(id, at_block), &validation_code.hash()); - ::PastCodePruning::put(&vec![(id, included_block)]); + PastCodeHash::::insert(&(id, at_block), &validation_code.hash()); + PastCodePruning::::put(&vec![(id, included_block)]); { let mut code_meta = Paras::past_code_meta(&id); code_meta.note_replacement(at_block, included_block); - ::PastCodeMeta::insert(&id, &code_meta); + PastCodeMeta::::insert(&id, &code_meta); } let pruned_at: BlockNumber = included_block + code_retention_period + 1; - assert_eq!( - ::PastCodeHash::get(&(id, at_block)), - Some(validation_code.hash()) - ); + assert_eq!(PastCodeHash::::get(&(id, at_block)), Some(validation_code.hash())); check_code_is_stored(&validation_code); run_to_block(pruned_at - 1, None); - assert_eq!( - ::PastCodeHash::get(&(id, at_block)), - Some(validation_code.hash()) - ); + assert_eq!(PastCodeHash::::get(&(id, at_block)), Some(validation_code.hash())); assert_eq!(Paras::past_code_meta(&id).most_recent_change(), Some(at_block)); check_code_is_stored(&validation_code); run_to_block(pruned_at, None); - assert!(::PastCodeHash::get(&(id, at_block)).is_none()); + assert!(PastCodeHash::::get(&(id, at_block)).is_none()); assert!(Paras::past_code_meta(&id).most_recent_change().is_none()); check_code_is_not_stored(&validation_code); }); @@ -377,7 +371,7 @@ fn note_past_code_sets_up_pruning_correctly() { Paras::note_past_code(id_a, 10, 12, ValidationCode(vec![1, 2, 3]).hash()); Paras::note_past_code(id_b, 20, 23, ValidationCode(vec![4, 5, 6]).hash()); - assert_eq!(::PastCodePruning::get(), vec![(id_a, 12), (id_b, 23)]); + assert_eq!(PastCodePruning::::get(), vec![(id_a, 12), (id_b, 23)]); assert_eq!( Paras::past_code_meta(&id_a), ParaPastCodeMeta { upgrade_times: vec![upgrade_at(10, 12)], last_pruned: None } @@ -437,13 +431,10 @@ fn code_upgrade_applied_after_delay() { Paras::note_new_head(para_id, Default::default(), 1); assert!(Paras::past_code_meta(¶_id).most_recent_change().is_none()); - assert_eq!(::FutureCodeUpgrades::get(¶_id), Some(expected_at)); - assert_eq!(::FutureCodeHash::get(¶_id), Some(new_code.hash())); - assert_eq!(::UpcomingUpgrades::get(), vec![(para_id, expected_at)]); - assert_eq!( - ::UpgradeCooldowns::get(), - vec![(para_id, next_possible_upgrade_at)] - ); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(expected_at)); + assert_eq!(FutureCodeHash::::get(¶_id), Some(new_code.hash())); + assert_eq!(UpcomingUpgrades::::get(), vec![(para_id, expected_at)]); + assert_eq!(UpgradeCooldowns::::get(), vec![(para_id, next_possible_upgrade_at)]); assert_eq!(Paras::current_code(¶_id), Some(original_code.clone())); check_code_is_stored(&original_code); check_code_is_stored(&new_code); @@ -459,12 +450,9 @@ fn code_upgrade_applied_after_delay() { Paras::note_new_head(para_id, Default::default(), expected_at - 1); assert!(Paras::past_code_meta(¶_id).most_recent_change().is_none()); - assert_eq!(::FutureCodeUpgrades::get(¶_id), Some(expected_at)); - assert_eq!(::FutureCodeHash::get(¶_id), Some(new_code.hash())); - assert_eq!( - ::UpgradeGoAheadSignal::get(¶_id), - Some(UpgradeGoAhead::GoAhead) - ); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(expected_at)); + assert_eq!(FutureCodeHash::::get(¶_id), Some(new_code.hash())); + assert_eq!(UpgradeGoAheadSignal::::get(¶_id), Some(UpgradeGoAhead::GoAhead)); assert_eq!(Paras::current_code(¶_id), Some(original_code.clone())); check_code_is_stored(&original_code); check_code_is_stored(&new_code); @@ -479,12 +467,12 @@ fn code_upgrade_applied_after_delay() { assert_eq!(Paras::past_code_meta(¶_id).most_recent_change(), Some(expected_at)); assert_eq!( - ::PastCodeHash::get(&(para_id, expected_at)), + PastCodeHash::::get(&(para_id, expected_at)), Some(original_code.hash()), ); - assert!(::FutureCodeUpgrades::get(¶_id).is_none()); - assert!(::FutureCodeHash::get(¶_id).is_none()); - assert!(::UpgradeGoAheadSignal::get(¶_id).is_none()); + assert!(FutureCodeUpgrades::::get(¶_id).is_none()); + assert!(FutureCodeHash::::get(¶_id).is_none()); + assert!(UpgradeGoAheadSignal::::get(¶_id).is_none()); assert_eq!(Paras::current_code(¶_id), Some(new_code.clone())); check_code_is_stored(&original_code); check_code_is_stored(&new_code); @@ -538,14 +526,11 @@ fn code_upgrade_applied_after_delay_even_when_late() { Paras::note_new_head(para_id, Default::default(), 1); assert!(Paras::past_code_meta(¶_id).most_recent_change().is_none()); - assert_eq!(::FutureCodeUpgrades::get(¶_id), Some(expected_at)); - assert_eq!(::FutureCodeHash::get(¶_id), Some(new_code.hash())); - assert_eq!(::UpcomingUpgrades::get(), vec![(para_id, expected_at)]); - assert_eq!( - ::UpgradeCooldowns::get(), - vec![(para_id, next_possible_upgrade_at)] - ); - assert!(::UpgradeGoAheadSignal::get(¶_id).is_none()); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(expected_at)); + assert_eq!(FutureCodeHash::::get(¶_id), Some(new_code.hash())); + assert_eq!(UpcomingUpgrades::::get(), vec![(para_id, expected_at)]); + assert_eq!(UpgradeCooldowns::::get(), vec![(para_id, next_possible_upgrade_at)]); + assert!(UpgradeGoAheadSignal::::get(¶_id).is_none()); assert_eq!(Paras::current_code(¶_id), Some(original_code.clone())); expected_at @@ -557,22 +542,19 @@ fn code_upgrade_applied_after_delay_even_when_late() { // the upgrade. { // The signal should be set to go-ahead until the new head is actually processed. - assert_eq!( - ::UpgradeGoAheadSignal::get(¶_id), - Some(UpgradeGoAhead::GoAhead), - ); + assert_eq!(UpgradeGoAheadSignal::::get(¶_id), Some(UpgradeGoAhead::GoAhead),); Paras::note_new_head(para_id, Default::default(), expected_at + 4); assert_eq!(Paras::past_code_meta(¶_id).most_recent_change(), Some(expected_at)); assert_eq!( - ::PastCodeHash::get(&(para_id, expected_at)), + PastCodeHash::::get(&(para_id, expected_at)), Some(original_code.hash()), ); - assert!(::FutureCodeUpgrades::get(¶_id).is_none()); - assert!(::FutureCodeHash::get(¶_id).is_none()); - assert!(::UpgradeGoAheadSignal::get(¶_id).is_none()); + assert!(FutureCodeUpgrades::::get(¶_id).is_none()); + assert!(FutureCodeHash::::get(¶_id).is_none()); + assert!(UpgradeGoAheadSignal::::get(¶_id).is_none()); assert_eq!(Paras::current_code(¶_id), Some(new_code.clone())); } }); @@ -615,11 +597,8 @@ fn submit_code_change_when_not_allowed_is_err() { run_to_block(1, None); Paras::schedule_code_upgrade(para_id, new_code.clone(), 1, &Configuration::config()); - assert_eq!( - ::FutureCodeUpgrades::get(¶_id), - Some(1 + validation_upgrade_delay) - ); - assert_eq!(::FutureCodeHash::get(¶_id), Some(new_code.hash())); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(1 + validation_upgrade_delay)); + assert_eq!(FutureCodeHash::::get(¶_id), Some(new_code.hash())); check_code_is_stored(&new_code); // We expect that if an upgrade is signalled while there is already one pending we just @@ -628,10 +607,10 @@ fn submit_code_change_when_not_allowed_is_err() { assert!(!Paras::can_upgrade_validation_code(para_id)); Paras::schedule_code_upgrade(para_id, newer_code.clone(), 2, &Configuration::config()); assert_eq!( - ::FutureCodeUpgrades::get(¶_id), + FutureCodeUpgrades::::get(¶_id), Some(1 + validation_upgrade_delay), // did not change since the same assertion from the last time. ); - assert_eq!(::FutureCodeHash::get(¶_id), Some(new_code.hash())); + assert_eq!(FutureCodeHash::::get(¶_id), Some(new_code.hash())); check_code_is_not_stored(&newer_code); }); } @@ -685,27 +664,21 @@ fn upgrade_restriction_elapsed_doesnt_mean_can_upgrade() { Paras::schedule_code_upgrade(para_id, new_code.clone(), 0, &Configuration::config()); Paras::note_new_head(para_id, dummy_head_data(), 0); assert_eq!( - ::UpgradeRestrictionSignal::get(¶_id), + UpgradeRestrictionSignal::::get(¶_id), Some(UpgradeRestriction::Present), ); - assert_eq!( - ::FutureCodeUpgrades::get(¶_id), - Some(0 + validation_upgrade_delay) - ); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(0 + validation_upgrade_delay)); assert!(!Paras::can_upgrade_validation_code(para_id)); run_to_block(31, None); - assert!(::UpgradeRestrictionSignal::get(¶_id).is_none()); + assert!(UpgradeRestrictionSignal::::get(¶_id).is_none()); // Note the para still cannot upgrade the validation code. assert!(!Paras::can_upgrade_validation_code(para_id)); // And scheduling another upgrade does not do anything. `expected_at` is still the same. Paras::schedule_code_upgrade(para_id, newer_code.clone(), 30, &Configuration::config()); - assert_eq!( - ::FutureCodeUpgrades::get(¶_id), - Some(0 + validation_upgrade_delay) - ); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(0 + validation_upgrade_delay)); }); } @@ -760,8 +733,8 @@ fn full_parachain_cleanup_storage() { Paras::note_new_head(para_id, Default::default(), 1); assert!(Paras::past_code_meta(¶_id).most_recent_change().is_none()); - assert_eq!(::FutureCodeUpgrades::get(¶_id), Some(expected_at)); - assert_eq!(::FutureCodeHash::get(¶_id), Some(new_code.hash())); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(expected_at)); + assert_eq!(FutureCodeHash::::get(¶_id), Some(new_code.hash())); assert_eq!(Paras::current_code(¶_id), Some(original_code.clone())); check_code_is_stored(&original_code); check_code_is_stored(&new_code); @@ -789,15 +762,15 @@ fn full_parachain_cleanup_storage() { // Why 7 and 8? See above, the clean up scheduled above was processed at the block 8. // The initial upgrade was enacted at the block 7. assert_eq!(Paras::past_code_meta(¶_id).most_recent_change(), Some(8)); - assert_eq!(::PastCodeHash::get(&(para_id, 8)), Some(new_code.hash())); - assert_eq!(::PastCodePruning::get(), vec![(para_id, 7), (para_id, 8)]); + assert_eq!(PastCodeHash::::get(&(para_id, 8)), Some(new_code.hash())); + assert_eq!(PastCodePruning::::get(), vec![(para_id, 7), (para_id, 8)]); check_code_is_stored(&original_code); check_code_is_stored(&new_code); // any future upgrades haven't been used to validate yet, so those // are cleaned up immediately. - assert!(::FutureCodeUpgrades::get(¶_id).is_none()); - assert!(::FutureCodeHash::get(¶_id).is_none()); + assert!(FutureCodeUpgrades::::get(¶_id).is_none()); + assert!(FutureCodeHash::::get(¶_id).is_none()); assert!(Paras::current_code(¶_id).is_none()); // run to do the final cleanup @@ -806,9 +779,9 @@ fn full_parachain_cleanup_storage() { // now the final cleanup: last past code cleaned up, and this triggers meta cleanup. assert_eq!(Paras::past_code_meta(¶_id), Default::default()); - assert!(::PastCodeHash::get(&(para_id, 7)).is_none()); - assert!(::PastCodeHash::get(&(para_id, 8)).is_none()); - assert!(::PastCodePruning::get().is_empty()); + assert!(PastCodeHash::::get(&(para_id, 7)).is_none()); + assert!(PastCodeHash::::get(&(para_id, 8)).is_none()); + assert!(PastCodePruning::::get().is_empty()); check_code_is_not_stored(&original_code); check_code_is_not_stored(&new_code); }); @@ -948,34 +921,34 @@ fn para_incoming_at_session() { }) .for_each(sign_and_include_pvf_check_statement); - assert_eq!(::ActionsQueue::get(Paras::scheduled_session()), vec![c, b, a],); + assert_eq!(ActionsQueue::::get(Paras::scheduled_session()), vec![c, b, a],); // Lifecycle is tracked correctly - assert_eq!(::ParaLifecycles::get(&a), Some(ParaLifecycle::Onboarding)); - assert_eq!(::ParaLifecycles::get(&b), Some(ParaLifecycle::Onboarding)); - assert_eq!(::ParaLifecycles::get(&c), Some(ParaLifecycle::Onboarding)); + assert_eq!(ParaLifecycles::::get(&a), Some(ParaLifecycle::Onboarding)); + assert_eq!(ParaLifecycles::::get(&b), Some(ParaLifecycle::Onboarding)); + assert_eq!(ParaLifecycles::::get(&c), Some(ParaLifecycle::Onboarding)); // run to block without session change. run_to_block(2, None); assert_eq!(Paras::parachains(), Vec::new()); - assert_eq!(::ActionsQueue::get(Paras::scheduled_session()), vec![c, b, a],); + assert_eq!(ActionsQueue::::get(Paras::scheduled_session()), vec![c, b, a],); // Lifecycle is tracked correctly - assert_eq!(::ParaLifecycles::get(&a), Some(ParaLifecycle::Onboarding)); - assert_eq!(::ParaLifecycles::get(&b), Some(ParaLifecycle::Onboarding)); - assert_eq!(::ParaLifecycles::get(&c), Some(ParaLifecycle::Onboarding)); + assert_eq!(ParaLifecycles::::get(&a), Some(ParaLifecycle::Onboarding)); + assert_eq!(ParaLifecycles::::get(&b), Some(ParaLifecycle::Onboarding)); + assert_eq!(ParaLifecycles::::get(&c), Some(ParaLifecycle::Onboarding)); // Two sessions pass, so action queue is triggered run_to_block(4, Some(vec![3, 4])); assert_eq!(Paras::parachains(), vec![c, b]); - assert_eq!(::ActionsQueue::get(Paras::scheduled_session()), Vec::new()); + assert_eq!(ActionsQueue::::get(Paras::scheduled_session()), Vec::new()); // Lifecycle is tracked correctly - assert_eq!(::ParaLifecycles::get(&a), Some(ParaLifecycle::Parathread)); - assert_eq!(::ParaLifecycles::get(&b), Some(ParaLifecycle::Parachain)); - assert_eq!(::ParaLifecycles::get(&c), Some(ParaLifecycle::Parachain)); + assert_eq!(ParaLifecycles::::get(&a), Some(ParaLifecycle::Parathread)); + assert_eq!(ParaLifecycles::::get(&b), Some(ParaLifecycle::Parachain)); + assert_eq!(ParaLifecycles::::get(&c), Some(ParaLifecycle::Parachain)); assert_eq!(Paras::current_code(&a), Some(vec![2].into())); assert_eq!(Paras::current_code(&b), Some(vec![1].into())); @@ -1054,18 +1027,18 @@ fn code_ref_is_cleaned_correctly() { Paras::increase_code_ref(&code.hash(), &code); Paras::increase_code_ref(&code.hash(), &code); - assert!(::CodeByHash::contains_key(code.hash())); - assert_eq!(::CodeByHashRefs::get(code.hash()), 2); + assert!(CodeByHash::::contains_key(code.hash())); + assert_eq!(CodeByHashRefs::::get(code.hash()), 2); Paras::decrease_code_ref(&code.hash()); - assert!(::CodeByHash::contains_key(code.hash())); - assert_eq!(::CodeByHashRefs::get(code.hash()), 1); + assert!(CodeByHash::::contains_key(code.hash())); + assert_eq!(CodeByHashRefs::::get(code.hash()), 1); Paras::decrease_code_ref(&code.hash()); - assert!(!::CodeByHash::contains_key(code.hash())); - assert!(!::CodeByHashRefs::contains_key(code.hash())); + assert!(!CodeByHash::::contains_key(code.hash())); + assert!(!CodeByHashRefs::::contains_key(code.hash())); }); } @@ -1140,11 +1113,11 @@ fn pvf_check_coalescing_onboarding_and_upgrade() { .for_each(sign_and_include_pvf_check_statement); // Check that `b` actually onboards. - assert_eq!(::ActionsQueue::get(EXPECTED_SESSION + 2), vec![b]); + assert_eq!(ActionsQueue::::get(EXPECTED_SESSION + 2), vec![b]); // Check that the upgrade got scheduled. assert_eq!( - ::FutureCodeUpgrades::get(&a), + FutureCodeUpgrades::::get(&a), Some(RELAY_PARENT + validation_upgrade_delay), ); @@ -1190,7 +1163,7 @@ fn pvf_check_onboarding_reject_on_expiry() { // Make sure that we kicked off the PVF vote for this validation code and that the // validation code is stored. - assert!(::PvfActiveVoteMap::get(&validation_code.hash()).is_some()); + assert!(PvfActiveVoteMap::::get(&validation_code.hash()).is_some()); check_code_is_stored(&validation_code); // Skip 2 sessions (i.e. `pvf_voting_ttl`) verifying that the code is still stored in @@ -1204,7 +1177,7 @@ fn pvf_check_onboarding_reject_on_expiry() { // Verify that the PVF is no longer stored and there is no active PVF vote. check_code_is_not_stored(&validation_code); - assert!(::PvfActiveVoteMap::get(&validation_code.hash()).is_none()); + assert!(PvfActiveVoteMap::::get(&validation_code.hash()).is_none()); assert!(Paras::pvfs_require_precheck().is_empty()); // Verify that at this point we can again try to initialize the same para. @@ -1271,9 +1244,9 @@ fn pvf_check_upgrade_reject() { // Verify that the new code is discarded. check_code_is_not_stored(&new_code); - assert!(::PvfActiveVoteMap::get(&new_code.hash()).is_none()); + assert!(PvfActiveVoteMap::::get(&new_code.hash()).is_none()); assert!(Paras::pvfs_require_precheck().is_empty()); - assert!(::FutureCodeHash::get(&a).is_none()); + assert!(FutureCodeHash::::get(&a).is_none()); // Verify that the required events were emitted. EventValidator::new().started(&new_code, a).rejected(&new_code, a).check(); @@ -1516,7 +1489,7 @@ fn add_trusted_validation_code_inserts_with_no_users() { RuntimeOrigin::root(), validation_code.clone() )); - assert_eq!(::CodeByHashRefs::get(&validation_code.hash()), 0,); + assert_eq!(CodeByHashRefs::::get(&validation_code.hash()), 0,); }); } @@ -1554,8 +1527,8 @@ fn poke_unused_validation_code_removes_code_cleanly() { validation_code.hash() )); - assert_eq!(::CodeByHashRefs::get(&validation_code.hash()), 0); - assert!(!::CodeByHash::contains_key(&validation_code.hash())); + assert_eq!(CodeByHashRefs::::get(&validation_code.hash()), 0); + assert!(!CodeByHash::::contains_key(&validation_code.hash())); }); } @@ -1598,15 +1571,15 @@ fn increase_code_ref_doesnt_have_allergy_on_add_trusted_validation_code() { assert_ok!(Paras::add_trusted_validation_code(RuntimeOrigin::root(), code.clone())); Paras::increase_code_ref(&code.hash(), &code); Paras::increase_code_ref(&code.hash(), &code); - assert!(::CodeByHash::contains_key(code.hash())); - assert_eq!(::CodeByHashRefs::get(code.hash()), 2); + assert!(CodeByHash::::contains_key(code.hash())); + assert_eq!(CodeByHashRefs::::get(code.hash()), 2); }); new_test_ext(Default::default()).execute_with(|| { assert_ok!(Paras::add_trusted_validation_code(RuntimeOrigin::root(), code.clone())); Paras::decrease_code_ref(&code.hash()); - assert!(::CodeByHash::contains_key(code.hash())); - assert_eq!(::CodeByHashRefs::get(code.hash()), 0); + assert!(CodeByHash::::contains_key(code.hash())); + assert_eq!(CodeByHashRefs::::get(code.hash()), 0); }); } @@ -1643,10 +1616,7 @@ fn add_trusted_validation_code_insta_approval() { // Verify that the code upgrade has `expected_at` set to `26`. This is the behavior // equal to that of `pvf_checking_enabled: false`. - assert_eq!( - ::FutureCodeUpgrades::get(¶_id), - Some(1 + validation_upgrade_delay) - ); + assert_eq!(FutureCodeUpgrades::::get(¶_id), Some(1 + validation_upgrade_delay)); // Verify that the required events were emitted. EventValidator::new() @@ -1685,16 +1655,16 @@ fn add_trusted_validation_code_enacts_existing_pvf_vote() { // No upgrade should be scheduled at this point. PVF pre-checking vote should run for // that PVF. - assert!(::FutureCodeUpgrades::get(¶_id).is_none()); - assert!(::PvfActiveVoteMap::contains_key(&validation_code.hash())); + assert!(FutureCodeUpgrades::::get(¶_id).is_none()); + assert!(PvfActiveVoteMap::::contains_key(&validation_code.hash())); // Then we add a trusted validation code. That should conclude the vote. assert_ok!(Paras::add_trusted_validation_code( RuntimeOrigin::root(), validation_code.clone() )); - assert!(::FutureCodeUpgrades::get(¶_id).is_some()); - assert!(!::PvfActiveVoteMap::contains_key(&validation_code.hash())); + assert!(FutureCodeUpgrades::::get(¶_id).is_some()); + assert!(!PvfActiveVoteMap::::contains_key(&validation_code.hash())); }); } @@ -1706,7 +1676,7 @@ fn verify_upgrade_go_ahead_signal_is_externally_accessible() { new_test_ext(Default::default()).execute_with(|| { assert!(sp_io::storage::get(&well_known_keys::upgrade_go_ahead_signal(a)).is_none()); - ::UpgradeGoAheadSignal::insert(&a, UpgradeGoAhead::GoAhead); + UpgradeGoAheadSignal::::insert(&a, UpgradeGoAhead::GoAhead); assert_eq!( sp_io::storage::get(&well_known_keys::upgrade_go_ahead_signal(a)).unwrap(), vec![1u8], @@ -1722,7 +1692,7 @@ fn verify_upgrade_restriction_signal_is_externally_accessible() { new_test_ext(Default::default()).execute_with(|| { assert!(sp_io::storage::get(&well_known_keys::upgrade_restriction_signal(a)).is_none()); - ::UpgradeRestrictionSignal::insert(&a, UpgradeRestriction::Present); + UpgradeRestrictionSignal::::insert(&a, UpgradeRestriction::Present); assert_eq!( sp_io::storage::get(&well_known_keys::upgrade_restriction_signal(a)).unwrap(), vec![0], @@ -1783,7 +1753,7 @@ fn parachains_cache_is_set() { drop(parachains_cache); // In order after addition - assert_eq!(::Parachains::get(), vec![a]); + assert_eq!(Parachains::::get(), vec![a]); let mut parachains_cache: ParachainsCache = ParachainsCache::new(); @@ -1795,14 +1765,14 @@ fn parachains_cache_is_set() { drop(parachains_cache); // In order after removal - assert_eq!(::Parachains::get(), vec![]); + assert_eq!(Parachains::::get(), vec![]); let mut parachains_cache: ParachainsCache = ParachainsCache::new(); // Remove nonexisting element parachains_cache.remove(a); assert_storage_noop!(drop(parachains_cache)); - assert_eq!(::Parachains::get(), vec![]); + assert_eq!(Parachains::::get(), vec![]); }); } @@ -1826,7 +1796,7 @@ fn parachains_cache_preserves_order() { drop(parachains_cache); // In order after addition - assert_eq!(::Parachains::get(), vec![a, b, c, d]); + assert_eq!(Parachains::::get(), vec![a, b, c, d]); let mut parachains_cache: ParachainsCache = ParachainsCache::new(); @@ -1838,6 +1808,6 @@ fn parachains_cache_preserves_order() { drop(parachains_cache); // In order after removal - assert_eq!(::Parachains::get(), vec![a, c]); + assert_eq!(Parachains::::get(), vec![a, c]); }); } diff --git a/runtime/parachains/src/paras_inherent/mod.rs b/runtime/parachains/src/paras_inherent/mod.rs index ee6666b0060b..db9caca49d1b 100644 --- a/runtime/parachains/src/paras_inherent/mod.rs +++ b/runtime/parachains/src/paras_inherent/mod.rs @@ -104,7 +104,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/parachains/src/runtime_api_impl/mod.rs b/runtime/parachains/src/runtime_api_impl/mod.rs index da1cd6d09337..e5a17f33f3ad 100644 --- a/runtime/parachains/src/runtime_api_impl/mod.rs +++ b/runtime/parachains/src/runtime_api_impl/mod.rs @@ -26,5 +26,5 @@ //! 2. Move methods from `vstaging` to `v3`. The new stable version should include //! all methods from `vstaging` tagged with the new version number (e.g. all //! `v3` methods). -pub mod v2; +pub mod v4; pub mod vstaging; diff --git a/runtime/parachains/src/runtime_api_impl/v2.rs b/runtime/parachains/src/runtime_api_impl/v4.rs similarity index 91% rename from runtime/parachains/src/runtime_api_impl/v2.rs rename to runtime/parachains/src/runtime_api_impl/v4.rs index 15b91d57d923..da34a0723a06 100644 --- a/runtime/parachains/src/runtime_api_impl/v2.rs +++ b/runtime/parachains/src/runtime_api_impl/v4.rs @@ -18,15 +18,16 @@ //! functions. use crate::{ - configuration, dmp, hrmp, inclusion, initializer, paras, paras_inherent, scheduler, + configuration, disputes, dmp, hrmp, inclusion, initializer, paras, paras_inherent, scheduler, session_info, shared, }; use primitives::{ - AuthorityDiscoveryId, CandidateEvent, CommittedCandidateReceipt, CoreIndex, CoreOccupied, - CoreState, GroupIndex, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, - InboundHrmpMessage, OccupiedCore, OccupiedCoreAssumption, PersistedValidationData, - PvfCheckStatement, ScheduledCore, ScrapedOnChainVotes, SessionIndex, SessionInfo, - ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, ValidatorSignature, + AuthorityDiscoveryId, CandidateEvent, CandidateHash, CommittedCandidateReceipt, CoreIndex, + CoreOccupied, CoreState, DisputeState, ExecutorParams, GroupIndex, GroupRotationInfo, Hash, + Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, OccupiedCore, OccupiedCoreAssumption, + PersistedValidationData, PvfCheckStatement, ScheduledCore, ScrapedOnChainVotes, SessionIndex, + SessionInfo, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, + ValidatorSignature, }; use sp_runtime::traits::One; use sp_std::{collections::btree_map::BTreeMap, prelude::*}; @@ -399,3 +400,24 @@ where >::current_code_hash(¶_id) }) } + +/// Implementation for `get_session_disputes` function from the runtime API +pub fn get_session_disputes( +) -> Vec<(SessionIndex, CandidateHash, DisputeState)> { + >::disputes() +} + +/// Get session executor parameter set +pub fn session_executor_params( + session_index: SessionIndex, +) -> Option { + // This is to bootstrap the storage working around the runtime migration issue: + // https://github.com/paritytech/substrate/issues/9997 + // After the bootstrap is complete (no less than 7 session passed with the runtime) + // this code should be replaced with a pure + // >::session_executor_params(session_index) call. + match >::session_executor_params(session_index) { + Some(ep) => Some(ep), + None => Some(ExecutorParams::default()), + } +} diff --git a/runtime/parachains/src/runtime_api_impl/vstaging.rs b/runtime/parachains/src/runtime_api_impl/vstaging.rs index f96323ce5043..c6ab554030a7 100644 --- a/runtime/parachains/src/runtime_api_impl/vstaging.rs +++ b/runtime/parachains/src/runtime_api_impl/vstaging.rs @@ -15,28 +15,3 @@ // along with Polkadot. If not, see . //! Put implementations of functions from staging APIs here. - -use crate::{disputes, session_info}; -use primitives::{vstaging::ExecutorParams, CandidateHash, DisputeState, SessionIndex}; -use sp_std::prelude::*; - -/// Implementation for `get_session_disputes` function from the runtime API -pub fn get_session_disputes( -) -> Vec<(SessionIndex, CandidateHash, DisputeState)> { - >::disputes() -} - -/// Get session executor parameter set -pub fn session_executor_params( - session_index: SessionIndex, -) -> Option { - // This is to bootstrap the storage working around the runtime migration issue: - // https://github.com/paritytech/substrate/issues/9997 - // After the bootstrap is complete (no less than 7 session passed with the runtime) - // this code should be replaced with a pure - // >::session_executor_params(session_index) call. - match >::session_executor_params(session_index) { - Some(ep) => Some(ep), - None => Some(ExecutorParams::default()), - } -} diff --git a/runtime/parachains/src/scheduler.rs b/runtime/parachains/src/scheduler.rs index 3c671b9e86e3..58509350a3a6 100644 --- a/runtime/parachains/src/scheduler.rs +++ b/runtime/parachains/src/scheduler.rs @@ -158,7 +158,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/parachains/src/scheduler/tests.rs b/runtime/parachains/src/scheduler/tests.rs index 1d679eca1b39..76bdc563d827 100644 --- a/runtime/parachains/src/scheduler/tests.rs +++ b/runtime/parachains/src/scheduler/tests.rs @@ -865,7 +865,7 @@ fn schedule_rotates_groups() { _ => None, }); - let session_start_block = ::SessionStartBlock::get(); + let session_start_block = SessionStartBlock::::get(); assert_eq!(session_start_block, 1); Scheduler::add_parathread_claim(ParathreadClaim(thread_a, collator.clone())); diff --git a/runtime/parachains/src/session_info.rs b/runtime/parachains/src/session_info.rs index 1105bc2e5023..df0dd5cc7874 100644 --- a/runtime/parachains/src/session_info.rs +++ b/runtime/parachains/src/session_info.rs @@ -28,8 +28,7 @@ use frame_support::{ traits::{OneSessionHandler, ValidatorSet, ValidatorSetWithIdentification}, }; use primitives::{ - vstaging::{ExecutorParam, ExecutorParams}, - AssignmentId, AuthorityDiscoveryId, SessionIndex, SessionInfo, + AssignmentId, AuthorityDiscoveryId, ExecutorParam, ExecutorParams, SessionIndex, SessionInfo, }; use sp_std::vec::Vec; @@ -63,7 +62,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::storage_version(migration::STORAGE_VERSION)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/parachains/src/shared.rs b/runtime/parachains/src/shared.rs index 70e23644d232..3cc59c32878d 100644 --- a/runtime/parachains/src/shared.rs +++ b/runtime/parachains/src/shared.rs @@ -43,7 +43,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/runtime/parachains/src/ump.rs b/runtime/parachains/src/ump.rs index 992392c8b500..0a6fc06db9b4 100644 --- a/runtime/parachains/src/ump.rs +++ b/runtime/parachains/src/ump.rs @@ -215,7 +215,6 @@ pub mod pallet { use super::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] #[pallet::storage_version(migration::STORAGE_VERSION)] pub struct Pallet(_); @@ -402,22 +401,20 @@ impl Pallet { /// Remove all relevant storage items for an outgoing parachain. pub(crate) fn clean_ump_after_outgoing(outgoing_para: &ParaId) -> Weight { - ::RelayDispatchQueueSize::remove(outgoing_para); - ::RelayDispatchQueues::remove(outgoing_para); + RelayDispatchQueueSize::::remove(outgoing_para); + RelayDispatchQueues::::remove(outgoing_para); // Remove the outgoing para from the `NeedsDispatch` list and from // `NextDispatchRoundStartWith`. // // That's needed for maintaining invariant that `NextDispatchRoundStartWith` points to an // existing item in `NeedsDispatch`. - ::NeedsDispatch::mutate(|v| { + NeedsDispatch::::mutate(|v| { if let Ok(i) = v.binary_search(outgoing_para) { v.remove(i); } }); - ::NextDispatchRoundStartWith::mutate(|v| { - *v = v.filter(|p| p == outgoing_para) - }); + NextDispatchRoundStartWith::::mutate(|v| *v = v.filter(|p| p == outgoing_para)); ::WeightInfo::clean_ump_after_outgoing() } @@ -436,8 +433,7 @@ impl Pallet { }) } - let (mut para_queue_count, mut para_queue_size) = - ::RelayDispatchQueueSize::get(¶); + let (mut para_queue_count, mut para_queue_size) = RelayDispatchQueueSize::::get(¶); for (idx, msg) in upward_messages.into_iter().enumerate() { let msg_size = msg.len() as u32; @@ -479,19 +475,14 @@ impl Pallet { .iter() .fold((0, 0), |(cnt, size), d| (cnt + 1, size + d.len() as u32)); - ::RelayDispatchQueues::mutate(¶, |v| { - v.extend(upward_messages.into_iter()) - }); + RelayDispatchQueues::::mutate(¶, |v| v.extend(upward_messages.into_iter())); - ::RelayDispatchQueueSize::mutate( - ¶, - |(ref mut cnt, ref mut size)| { - *cnt += extra_count; - *size += extra_size; - }, - ); + RelayDispatchQueueSize::::mutate(¶, |(ref mut cnt, ref mut size)| { + *cnt += extra_count; + *size += extra_size; + }); - ::NeedsDispatch::mutate(|v| { + NeedsDispatch::::mutate(|v| { if let Err(i) = v.binary_search(¶) { v.insert(i, para); } @@ -592,13 +583,13 @@ impl Pallet { /// Puts a given upward message into the list of overweight messages allowing it to be executed /// later. fn stash_overweight(sender: ParaId, upward_message: Vec) -> OverweightIndex { - let index = ::OverweightCount::mutate(|count| { + let index = OverweightCount::::mutate(|count| { let index = *count; *count += 1; index }); - ::Overweight::insert(index, (sender, upward_message)); + Overweight::::insert(index, (sender, upward_message)); index } } @@ -714,8 +705,8 @@ struct NeedsDispatchCursor { impl NeedsDispatchCursor { fn new() -> Self { - let needs_dispatch: Vec = as Store>::NeedsDispatch::get(); - let start_with = as Store>::NextDispatchRoundStartWith::get(); + let needs_dispatch: Vec = NeedsDispatch::::get(); + let start_with = NextDispatchRoundStartWith::::get(); let initial_index = match start_with { Some(para) => match needs_dispatch.binary_search(¶) { @@ -766,7 +757,7 @@ impl NeedsDispatchCursor { /// Flushes the dispatcher state into the persistent storage. fn flush(self) { let next_one = self.peek(); - as Store>::NextDispatchRoundStartWith::set(next_one); - as Store>::NeedsDispatch::put(self.needs_dispatch); + NextDispatchRoundStartWith::::set(next_one); + NeedsDispatch::::put(self.needs_dispatch); } } diff --git a/runtime/parachains/src/ump/tests.rs b/runtime/parachains/src/ump/tests.rs index 39f78c8eba99..60cc02ff0ff7 100644 --- a/runtime/parachains/src/ump/tests.rs +++ b/runtime/parachains/src/ump/tests.rs @@ -79,13 +79,13 @@ fn queue_upward_msg(para: ParaId, msg: UpwardMessage) { fn assert_storage_consistency_exhaustive() { // check that empty queues don't clutter the storage. - for (_para, queue) in ::RelayDispatchQueues::iter() { + for (_para, queue) in RelayDispatchQueues::::iter() { assert!(!queue.is_empty()); } // actually count the counts and sizes in queues and compare them to the bookkept version. - for (para, queue) in ::RelayDispatchQueues::iter() { - let (expected_count, expected_size) = ::RelayDispatchQueueSize::get(para); + for (para, queue) in RelayDispatchQueues::::iter() { + let (expected_count, expected_size) = RelayDispatchQueueSize::::get(para); let (actual_count, actual_size) = queue .into_iter() .fold((0, 0), |(acc_count, acc_size), x| (acc_count + 1, acc_size + x.len() as u32)); @@ -96,24 +96,22 @@ fn assert_storage_consistency_exhaustive() { // since we wipe the empty queues the sets of paras in queue contents, queue sizes and // need dispatch set should all be equal. - let queue_contents_set = ::RelayDispatchQueues::iter() + let queue_contents_set = + RelayDispatchQueues::::iter().map(|(k, _)| k).collect::>(); + let queue_sizes_set = RelayDispatchQueueSize::::iter() .map(|(k, _)| k) .collect::>(); - let queue_sizes_set = ::RelayDispatchQueueSize::iter() - .map(|(k, _)| k) - .collect::>(); - let needs_dispatch_set = - ::NeedsDispatch::get().into_iter().collect::>(); + let needs_dispatch_set = NeedsDispatch::::get().into_iter().collect::>(); assert_eq!(queue_contents_set, queue_sizes_set); assert_eq!(queue_contents_set, needs_dispatch_set); // `NextDispatchRoundStartWith` should point into a para that is tracked. - if let Some(para) = ::NextDispatchRoundStartWith::get() { + if let Some(para) = NextDispatchRoundStartWith::::get() { assert!(queue_contents_set.contains(¶)); } // `NeedsDispatch` is always sorted. - assert!(::NeedsDispatch::get().windows(2).all(|xs| xs[0] <= xs[1])); + assert!(NeedsDispatch::::get().windows(2).all(|xs| xs[0] <= xs[1])); } #[test] diff --git a/runtime/polkadot/src/lib.rs b/runtime/polkadot/src/lib.rs index b5ddb16a2800..f33990348aa4 100644 --- a/runtime/polkadot/src/lib.rs +++ b/runtime/polkadot/src/lib.rs @@ -31,7 +31,7 @@ use runtime_parachains::{ dmp as parachains_dmp, hrmp as parachains_hrmp, inclusion as parachains_inclusion, initializer as parachains_initializer, origin as parachains_origin, paras as parachains_paras, paras_inherent as parachains_paras_inherent, reward_points as parachains_reward_points, - runtime_api_impl::v2 as parachains_runtime_api_impl, scheduler as parachains_scheduler, + runtime_api_impl::v4 as parachains_runtime_api_impl, scheduler as parachains_scheduler, session_info as parachains_session_info, shared as parachains_shared, ump as parachains_ump, }; @@ -41,24 +41,24 @@ use frame_election_provider_support::{generate_solution_type, onchain, Sequentia use frame_support::{ construct_runtime, parameter_types, traits::{ - ConstU32, EitherOfDiverse, InstanceFilter, KeyOwnerProofSystem, LockIdentifier, + ConstU32, EitherOf, EitherOfDiverse, InstanceFilter, KeyOwnerProofSystem, LockIdentifier, PrivilegeCmp, WithdrawReasons, }, weights::ConstantMultiplier, PalletId, RuntimeDebug, }; -use frame_system::EnsureRoot; +use frame_system::{EnsureRoot, EnsureWithSuccess}; use pallet_grandpa::{fg_primitives, AuthorityId as GrandpaId}; use pallet_im_online::sr25519::AuthorityId as ImOnlineId; use pallet_session::historical as session_historical; use pallet_transaction_payment::{FeeDetails, RuntimeDispatchInfo}; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CommittedCandidateReceipt, - CoreState, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, - Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, - SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, - LOWEST_PUBLIC_ID, + AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, + Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, + OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionInfo, Signature, + ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, LOWEST_PUBLIC_ID, }; use sp_core::OpaqueMetadata; use sp_mmr_primitives as mmr; @@ -114,13 +114,13 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { spec_name: create_runtime_str!("polkadot"), impl_name: create_runtime_str!("parity-polkadot"), authoring_version: 0, - spec_version: 9370, + spec_version: 9390, impl_version: 0, #[cfg(not(feature = "disable-runtime-api"))] apis: RUNTIME_API_VERSIONS, #[cfg(feature = "disable-runtime-api")] apis: sp_version::create_apis_vec![[]], - transaction_version: 20, + transaction_version: 21, state_version: 0, }; @@ -426,6 +426,7 @@ impl pallet_election_provider_multi_phase::MinerConfig for Runtime { as frame_election_provider_support::ElectionDataProvider >::MaxVotesPerVoter; + type MaxWinners = MaxActiveValidators; // The unsigned submissions have to respect the weight of the submit_unsigned call, thus their // weight estimate function is wired to this call's weight. @@ -826,6 +827,7 @@ parameter_types! { pub const MaxPeerInHeartbeats: u32 = 10_000; pub const MaxPeerDataEncodingSize: u32 = 1_000; pub const RootSpendOriginMaxAmount: Balance = Balance::MAX; + pub const CouncilSpendOriginMaxAmount: Balance = Balance::MAX; } type ApproveOrigin = EitherOfDiverse< @@ -849,7 +851,14 @@ impl pallet_treasury::Config for Runtime { type SpendFunds = Bounties; type MaxApprovals = MaxApprovals; type WeightInfo = weights::pallet_treasury::WeightInfo; - type SpendOrigin = frame_system::EnsureRootWithSuccess; + type SpendOrigin = EitherOf< + frame_system::EnsureRootWithSuccess, + EnsureWithSuccess< + pallet_collective::EnsureProportionAtLeast, + AccountId, + CouncilSpendOriginMaxAmount, + >, + >; } parameter_types! { @@ -1576,17 +1585,22 @@ pub type SignedExtra = ( claims::PrevalidateAttests, ); -pub struct StakingMigrationV11OldPallet; -impl Get<&'static str> for StakingMigrationV11OldPallet { - fn get() -> &'static str { - "VoterList" +pub struct NominationPoolsMigrationV4OldPallet; +impl Get for NominationPoolsMigrationV4OldPallet { + fn get() -> Perbill { + Perbill::zero() } } /// All migrations that will run on the next runtime upgrade. /// /// Should be cleared after every release. -pub type Migrations = (); +pub type Migrations = ( + pallet_nomination_pools::migration::v4::MigrateToV4< + Runtime, + NominationPoolsMigrationV4OldPallet, + >, +); /// Unchecked extrinsic type as expected by this runtime. pub type UncheckedExtrinsic = @@ -1808,6 +1822,10 @@ sp_api::impl_runtime_apis! { parachains_runtime_api_impl::session_info::(index) } + fn session_executor_params(session_index: SessionIndex) -> Option { + parachains_runtime_api_impl::session_executor_params::(session_index) + } + fn dmq_contents(recipient: ParaId) -> Vec> { parachains_runtime_api_impl::dmq_contents::(recipient) } @@ -1842,6 +1860,10 @@ sp_api::impl_runtime_apis! { { parachains_runtime_api_impl::validation_code_hash::(para_id, assumption) } + + fn disputes() -> Vec<(SessionIndex, CandidateHash, DisputeState)> { + parachains_runtime_api_impl::get_session_disputes::() + } } impl beefy_primitives::BeefyApi for Runtime { @@ -2462,7 +2484,7 @@ mod multiplier_tests { #[cfg(all(test, feature = "try-runtime"))] mod remote_tests { use super::*; - use frame_try_runtime::{runtime_decl_for_TryRuntime::TryRuntime, UpgradeCheckSelect}; + use frame_try_runtime::{runtime_decl_for_try_runtime::TryRuntime, UpgradeCheckSelect}; use remote_externalities::{ Builder, Mode, OfflineConfig, OnlineConfig, SnapshotConfig, Transport, }; diff --git a/runtime/polkadot/src/weights/pallet_nomination_pools.rs b/runtime/polkadot/src/weights/pallet_nomination_pools.rs index 5baec28d664f..6e67e2854f17 100644 --- a/runtime/polkadot/src/weights/pallet_nomination_pools.rs +++ b/runtime/polkadot/src/weights/pallet_nomination_pools.rs @@ -16,22 +16,24 @@ //! Autogenerated weights for `pallet_nomination_pools` //! //! THIS FILE WAS AUTO-GENERATED USING THE SUBSTRATE BENCHMARK CLI VERSION 4.0.0-dev -//! DATE: 2023-02-27, STEPS: `50`, REPEAT: `20`, LOW RANGE: `[]`, HIGH RANGE: `[]` +//! DATE: 2023-03-06, STEPS: `50`, REPEAT: `20`, LOW RANGE: `[]`, HIGH RANGE: `[]` //! WORST CASE MAP SIZE: `1000000` -//! HOSTNAME: `bm4`, CPU: `Intel(R) Core(TM) i7-7700K CPU @ 4.20GHz` +//! HOSTNAME: `bm3`, CPU: `Intel(R) Core(TM) i7-7700K CPU @ 4.20GHz` //! EXECUTION: Some(Wasm), WASM-EXECUTION: Compiled, CHAIN: Some("polkadot-dev"), DB CACHE: 1024 // Executed Command: -// ./target/production/polkadot +// target/production/polkadot // benchmark // pallet -// --chain=polkadot-dev // --steps=50 // --repeat=20 -// --pallet=pallet_nomination_pools // --extrinsic=* // --execution=wasm // --wasm-execution=compiled +// --heap-pages=4096 +// --json-file=/var/lib/gitlab-runner/builds/zyw4fam_/0/parity/mirrors/polkadot/.git/.artifacts/bench.json +// --pallet=pallet_nomination_pools +// --chain=polkadot-dev // --header=./file_header.txt // --output=./runtime/polkadot/src/weights/ @@ -50,13 +52,15 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:2 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: NominationPools MaxPoolMembersPerPool (r:1 w:0) @@ -73,20 +77,22 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn join() -> Weight { // Proof Size summary in bytes: - // Measured: `3468` - // Estimated: `38468` - // Minimum execution time: 153_784 nanoseconds. - Weight::from_parts(155_121_000, 0) - .saturating_add(Weight::from_parts(0, 38468)) - .saturating_add(T::DbWeight::get().reads(17)) + // Measured: `3545` + // Estimated: `52915` + // Minimum execution time: 154_178_000 picoseconds. + Weight::from_parts(155_912_000, 0) + .saturating_add(Weight::from_parts(0, 52915)) + .saturating_add(T::DbWeight::get().reads(18)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:3 w:2) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -101,12 +107,12 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn bond_extra_transfer() -> Weight { // Proof Size summary in bytes: - // Measured: `3510` - // Estimated: `39063` - // Minimum execution time: 151_609 nanoseconds. - Weight::from_parts(155_385_000, 0) - .saturating_add(Weight::from_parts(0, 39063)) - .saturating_add(T::DbWeight::get().reads(14)) + // Measured: `3587` + // Estimated: `49550` + // Minimum execution time: 151_694_000 picoseconds. + Weight::from_parts(156_751_000, 0) + .saturating_add(Weight::from_parts(0, 49550)) + .saturating_add(T::DbWeight::get().reads(15)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools ClaimPermissions (r:1 w:0) @@ -114,9 +120,11 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:3 w:3) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -131,12 +139,12 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn bond_extra_other() -> Weight { // Proof Size summary in bytes: - // Measured: `3393` - // Estimated: `38950` - // Minimum execution time: 165_036 nanoseconds. - Weight::from_parts(166_264_000, 0) - .saturating_add(Weight::from_parts(0, 38950)) - .saturating_add(T::DbWeight::get().reads(14)) + // Measured: `3470` + // Estimated: `50427` + // Minimum execution time: 164_123_000 picoseconds. + Weight::from_parts(165_444_000, 0) + .saturating_add(Weight::from_parts(0, 50427)) + .saturating_add(T::DbWeight::get().reads(15)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools ClaimPermissions (r:1 w:0) @@ -144,31 +152,35 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:1 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) fn claim_payout() -> Weight { // Proof Size summary in bytes: - // Measured: `1220` - // Estimated: `13485` - // Minimum execution time: 61_052 nanoseconds. - Weight::from_parts(61_773_000, 0) - .saturating_add(Weight::from_parts(0, 13485)) - .saturating_add(T::DbWeight::get().reads(5)) + // Measured: `1297` + // Estimated: `20012` + // Minimum execution time: 61_194_000 picoseconds. + Weight::from_parts(62_067_000, 0) + .saturating_add(Weight::from_parts(0, 20012)) + .saturating_add(T::DbWeight::get().reads(6)) .saturating_add(T::DbWeight::get().writes(4)) } /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:2 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking CurrentEra (r:1 w:0) @@ -189,16 +201,16 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: NominationPools CounterForSubPoolsStorage (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn unbond() -> Weight { // Proof Size summary in bytes: - // Measured: `3748` - // Estimated: `44674` - // Minimum execution time: 155_226 nanoseconds. - Weight::from_parts(155_893_000, 0) - .saturating_add(Weight::from_parts(0, 44674)) - .saturating_add(T::DbWeight::get().reads(18)) + // Measured: `3825` + // Estimated: `60111` + // Minimum execution time: 155_272_000 picoseconds. + Weight::from_parts(156_629_000, 0) + .saturating_add(Weight::from_parts(0, 60111)) + .saturating_add(T::DbWeight::get().reads(19)) .saturating_add(T::DbWeight::get().writes(13)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) @@ -210,13 +222,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `s` is `[0, 100]`. fn pool_withdraw_unbonded(s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `1672` - // Estimated: `13025` - // Minimum execution time: 53_472 nanoseconds. - Weight::from_parts(55_368_257, 0) - .saturating_add(Weight::from_parts(0, 13025)) - // Standard Error: 904 - .saturating_add(Weight::from_parts(4_928, 0).saturating_mul(s.into())) + // Measured: `1676` + // Estimated: `18031` + // Minimum execution time: 54_207_000 picoseconds. + Weight::from_parts(56_142_332, 0) + .saturating_add(Weight::from_parts(0, 18031)) + // Standard Error: 4_547 + .saturating_add(Weight::from_parts(4_570, 0).saturating_mul(s.into())) .saturating_add(T::DbWeight::get().reads(5)) .saturating_add(T::DbWeight::get().writes(2)) } @@ -225,7 +237,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Staking CurrentEra (r:1 w:0) /// Proof: Staking CurrentEra (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools SubPoolsStorage (r:1 w:1) /// Proof: NominationPools SubPoolsStorage (max_values: None, max_size: Some(1197), added: 3672, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -243,13 +255,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `s` is `[0, 100]`. fn withdraw_unbonded_update(s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `2196` - // Estimated: `22991` - // Minimum execution time: 103_698 nanoseconds. - Weight::from_parts(105_075_914, 0) - .saturating_add(Weight::from_parts(0, 22991)) - // Standard Error: 1_503 - .saturating_add(Weight::from_parts(19_668, 0).saturating_mul(s.into())) + // Measured: `2200` + // Estimated: `31957` + // Minimum execution time: 103_172_000 picoseconds. + Weight::from_parts(106_375_310, 0) + .saturating_add(Weight::from_parts(0, 31957)) + // Standard Error: 6_965 + .saturating_add(Weight::from_parts(7_411, 0).saturating_mul(s.into())) .saturating_add(T::DbWeight::get().reads(9)) .saturating_add(T::DbWeight::get().writes(8)) } @@ -258,7 +270,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Staking CurrentEra (r:1 w:0) /// Proof: Staking CurrentEra (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools SubPoolsStorage (r:1 w:1) /// Proof: NominationPools SubPoolsStorage (max_values: None, max_size: Some(1197), added: 3672, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:1) @@ -282,7 +294,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools CounterForReversePoolIdLookup (r:1 w:1) /// Proof: NominationPools CounterForReversePoolIdLookup (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: NominationPools CounterForRewardPools (r:1 w:1) /// Proof: NominationPools CounterForRewardPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools CounterForSubPoolsStorage (r:1 w:1) @@ -298,11 +310,11 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `s` is `[0, 100]`. fn withdraw_unbonded_kill(_s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `2584` - // Estimated: `46001` - // Minimum execution time: 163_337 nanoseconds. - Weight::from_parts(167_095_416, 0) - .saturating_add(Weight::from_parts(0, 46001)) + // Measured: `2588` + // Estimated: `64903` + // Minimum execution time: 162_253_000 picoseconds. + Weight::from_parts(180_162_421, 0) + .saturating_add(Weight::from_parts(0, 64903)) .saturating_add(T::DbWeight::get().reads(20)) .saturating_add(T::DbWeight::get().writes(18)) } @@ -337,7 +349,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Balances Locks (r:1 w:1) /// Proof: Balances Locks (max_values: None, max_size: Some(1299), added: 3774, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: NominationPools CounterForRewardPools (r:1 w:1) /// Proof: NominationPools CounterForRewardPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools ReversePoolIdLookup (r:1 w:1) @@ -345,21 +357,21 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools CounterForReversePoolIdLookup (r:1 w:1) /// Proof: NominationPools CounterForReversePoolIdLookup (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Payee (r:0 w:1) /// Proof: Staking Payee (max_values: None, max_size: Some(73), added: 2548, mode: MaxEncodedLen) fn create() -> Weight { // Proof Size summary in bytes: // Measured: `1220` - // Estimated: `32002` - // Minimum execution time: 149_656 nanoseconds. - Weight::from_parts(150_555_000, 0) - .saturating_add(Weight::from_parts(0, 32002)) + // Estimated: `51890` + // Minimum execution time: 143_473_000 picoseconds. + Weight::from_parts(145_473_000, 0) + .saturating_add(Weight::from_parts(0, 51890)) .saturating_add(T::DbWeight::get().reads(21)) .saturating_add(T::DbWeight::get().writes(15)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) @@ -385,36 +397,36 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `n` is `[1, 16]`. fn nominate(n: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `1805` - // Estimated: `21998 + n * (2520 ±0)` - // Minimum execution time: 66_157 nanoseconds. - Weight::from_parts(66_972_809, 0) - .saturating_add(Weight::from_parts(0, 21998)) - // Standard Error: 9_409 - .saturating_add(Weight::from_parts(1_327_035, 0).saturating_mul(n.into())) + // Measured: `1809` + // Estimated: `33934 + n * (2520 ±0)` + // Minimum execution time: 67_193_000 picoseconds. + Weight::from_parts(68_165_415, 0) + .saturating_add(Weight::from_parts(0, 33934)) + // Standard Error: 10_479 + .saturating_add(Weight::from_parts(1_263_947, 0).saturating_mul(n.into())) .saturating_add(T::DbWeight::get().reads(12)) .saturating_add(T::DbWeight::get().reads((1_u64).saturating_mul(n.into()))) .saturating_add(T::DbWeight::get().writes(5)) .saturating_add(Weight::from_parts(0, 2520).saturating_mul(n.into())) } /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) fn set_state() -> Weight { // Proof Size summary in bytes: - // Measured: `1393` - // Estimated: `8752` - // Minimum execution time: 34_743 nanoseconds. - Weight::from_parts(35_038_000, 0) - .saturating_add(Weight::from_parts(0, 8752)) + // Measured: `1397` + // Estimated: `11778` + // Minimum execution time: 33_972_000 picoseconds. + Weight::from_parts(34_300_000, 0) + .saturating_add(Weight::from_parts(0, 11778)) .saturating_add(T::DbWeight::get().reads(3)) .saturating_add(T::DbWeight::get().writes(1)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools Metadata (r:1 w:1) /// Proof: NominationPools Metadata (max_values: None, max_size: Some(270), added: 2745, mode: MaxEncodedLen) /// Storage: NominationPools CounterForMetadata (r:1 w:1) @@ -422,13 +434,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `n` is `[1, 256]`. fn set_metadata(n: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `525` - // Estimated: `5883` - // Minimum execution time: 14_305 nanoseconds. - Weight::from_parts(14_805_073, 0) - .saturating_add(Weight::from_parts(0, 5883)) - // Standard Error: 125 - .saturating_add(Weight::from_parts(1_944, 0).saturating_mul(n.into())) + // Measured: `529` + // Estimated: `8909` + // Minimum execution time: 15_429_000 picoseconds. + Weight::from_parts(16_414_594, 0) + .saturating_add(Weight::from_parts(0, 8909)) + // Standard Error: 857 + .saturating_add(Weight::from_parts(732, 0).saturating_mul(n.into())) .saturating_add(T::DbWeight::get().reads(3)) .saturating_add(T::DbWeight::get().writes(2)) } @@ -440,31 +452,33 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: NominationPools MaxPoolMembersPerPool (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools MinCreateBond (r:0 w:1) /// Proof: NominationPools MinCreateBond (max_values: Some(1), max_size: Some(16), added: 511, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:0 w:1) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools MaxPools (r:0 w:1) /// Proof: NominationPools MaxPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn set_configs() -> Weight { // Proof Size summary in bytes: // Measured: `0` // Estimated: `0` - // Minimum execution time: 5_054 nanoseconds. - Weight::from_parts(5_333_000, 0) + // Minimum execution time: 6_354_000 picoseconds. + Weight::from_parts(6_629_000, 0) .saturating_add(Weight::from_parts(0, 0)) - .saturating_add(T::DbWeight::get().writes(5)) + .saturating_add(T::DbWeight::get().writes(6)) } /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) fn update_roles() -> Weight { // Proof Size summary in bytes: - // Measured: `525` - // Estimated: `2639` - // Minimum execution time: 20_025 nanoseconds. - Weight::from_parts(20_643_000, 0) - .saturating_add(Weight::from_parts(0, 2639)) + // Measured: `529` + // Estimated: `3685` + // Minimum execution time: 21_124_000 picoseconds. + Weight::from_parts(21_372_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) .saturating_add(T::DbWeight::get().reads(1)) .saturating_add(T::DbWeight::get().writes(1)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) @@ -483,14 +497,56 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList CounterForListNodes (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn chill() -> Weight { // Proof Size summary in bytes: - // Measured: `2032` - // Estimated: `20489` - // Minimum execution time: 65_418 nanoseconds. - Weight::from_parts(65_853_000, 0) - .saturating_add(Weight::from_parts(0, 20489)) + // Measured: `2036` + // Estimated: `29455` + // Minimum execution time: 65_370_000 picoseconds. + Weight::from_parts(66_126_000, 0) + .saturating_add(Weight::from_parts(0, 29455)) .saturating_add(T::DbWeight::get().reads(9)) .saturating_add(T::DbWeight::get().writes(5)) } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + /// Storage: NominationPools RewardPools (r:1 w:1) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) + /// Storage: System Account (r:1 w:0) + /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) + fn set_commission() -> Weight { + // Proof Size summary in bytes: + // Measured: `832` + // Estimated: `12324` + // Minimum execution time: 33_108_000 picoseconds. + Weight::from_parts(33_466_000, 0) + .saturating_add(Weight::from_parts(0, 12324)) + .saturating_add(T::DbWeight::get().reads(4)) + .saturating_add(T::DbWeight::get().writes(2)) + } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + fn set_commission_max() -> Weight { + // Proof Size summary in bytes: + // Measured: `569` + // Estimated: `3685` + // Minimum execution time: 19_755_000 picoseconds. + Weight::from_parts(20_033_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) + .saturating_add(T::DbWeight::get().reads(1)) + .saturating_add(T::DbWeight::get().writes(1)) + } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + fn set_commission_change_rate() -> Weight { + // Proof Size summary in bytes: + // Measured: `529` + // Estimated: `3685` + // Minimum execution time: 20_908_000 picoseconds. + Weight::from_parts(21_219_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) + .saturating_add(T::DbWeight::get().reads(1)) + .saturating_add(T::DbWeight::get().writes(1)) + } /// Storage: NominationPools PoolMembers (r:1 w:0) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools ClaimPermissions (r:1 w:1) @@ -498,11 +554,29 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo fn set_claim_permission() -> Weight { // Proof Size summary in bytes: // Measured: `508` - // Estimated: `5708` - // Minimum execution time: 14_918 nanoseconds. - Weight::from_parts(15_304_000, 0) - .saturating_add(Weight::from_parts(0, 5708)) + // Estimated: `7688` + // Minimum execution time: 15_851_000 picoseconds. + Weight::from_parts(16_068_000, 0) + .saturating_add(Weight::from_parts(0, 7688)) .saturating_add(T::DbWeight::get().reads(2)) .saturating_add(T::DbWeight::get().writes(1)) } + /// Storage: NominationPools BondedPools (r:1 w:0) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + /// Storage: NominationPools RewardPools (r:1 w:1) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) + /// Storage: System Account (r:1 w:1) + /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) + fn claim_commission() -> Weight { + // Proof Size summary in bytes: + // Measured: `1062` + // Estimated: `12324` + // Minimum execution time: 44_544_000 picoseconds. + Weight::from_parts(45_240_000, 0) + .saturating_add(Weight::from_parts(0, 12324)) + .saturating_add(T::DbWeight::get().reads(4)) + .saturating_add(T::DbWeight::get().writes(2)) + } } diff --git a/runtime/rococo/src/lib.rs b/runtime/rococo/src/lib.rs index c48bf9af9ef3..b6ed0a0660f0 100644 --- a/runtime/rococo/src/lib.rs +++ b/runtime/rococo/src/lib.rs @@ -23,8 +23,8 @@ use pallet_nis::WithMaximumOf; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - vstaging::ExecutorParams, AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, - CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, GroupRotationInfo, Hash, + AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, @@ -38,15 +38,12 @@ use sp_std::{cmp::Ordering, collections::btree_map::BTreeMap, prelude::*}; use runtime_parachains::{ configuration as parachains_configuration, disputes as parachains_disputes, - disputes::slashing as parachains_slashing, - dmp as parachains_dmp, hrmp as parachains_hrmp, inclusion as parachains_inclusion, - initializer as parachains_initializer, origin as parachains_origin, paras as parachains_paras, + disputes::slashing as parachains_slashing, dmp as parachains_dmp, hrmp as parachains_hrmp, + inclusion as parachains_inclusion, initializer as parachains_initializer, + origin as parachains_origin, paras as parachains_paras, paras_inherent as parachains_paras_inherent, - runtime_api_impl::{ - v2 as parachains_runtime_api_impl, vstaging as parachains_runtime_api_impl_staging, - }, - scheduler as parachains_scheduler, session_info as parachains_session_info, - shared as parachains_shared, ump as parachains_ump, + runtime_api_impl::v4 as parachains_runtime_api_impl, scheduler as parachains_scheduler, + session_info as parachains_session_info, shared as parachains_shared, ump as parachains_ump, }; use authority_discovery_primitives::AuthorityId as AuthorityDiscoveryId; @@ -112,13 +109,13 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { spec_name: create_runtime_str!("rococo"), impl_name: create_runtime_str!("parity-rococo-v2.0"), authoring_version: 0, - spec_version: 9370, + spec_version: 9390, impl_version: 0, #[cfg(not(feature = "disable-runtime-api"))] apis: RUNTIME_API_VERSIONS, #[cfg(feature = "disable-runtime-api")] apis: sp_version::create_apis_vec![[]], - transaction_version: 18, + transaction_version: 19, state_version: 1, }; @@ -1625,7 +1622,6 @@ sp_api::impl_runtime_apis! { } } - #[api_version(4)] impl primitives::runtime_api::ParachainHost for Runtime { fn validators() -> Vec { parachains_runtime_api_impl::validators::() @@ -1690,7 +1686,7 @@ sp_api::impl_runtime_apis! { } fn session_executor_params(session_index: SessionIndex) -> Option { - parachains_runtime_api_impl_staging::session_executor_params::(session_index) + parachains_runtime_api_impl::session_executor_params::(session_index) } fn dmq_contents(recipient: ParaId) -> Vec> { @@ -1729,7 +1725,7 @@ sp_api::impl_runtime_apis! { } fn disputes() -> Vec<(SessionIndex, CandidateHash, DisputeState)> { - runtime_parachains::runtime_api_impl::vstaging::get_session_disputes::() + parachains_runtime_api_impl::get_session_disputes::() } } @@ -2117,7 +2113,7 @@ sp_api::impl_runtime_apis! { #[cfg(all(test, feature = "try-runtime"))] mod remote_tests { use super::*; - use frame_try_runtime::{runtime_decl_for_TryRuntime::TryRuntime, UpgradeCheckSelect}; + use frame_try_runtime::{runtime_decl_for_try_runtime::TryRuntime, UpgradeCheckSelect}; use remote_externalities::{ Builder, Mode, OfflineConfig, OnlineConfig, SnapshotConfig, Transport, }; diff --git a/runtime/test-runtime/src/lib.rs b/runtime/test-runtime/src/lib.rs index 35ec219f4685..4cdbbed04433 100644 --- a/runtime/test-runtime/src/lib.rs +++ b/runtime/test-runtime/src/lib.rs @@ -28,7 +28,7 @@ use polkadot_runtime_parachains::{ configuration as parachains_configuration, disputes as parachains_disputes, dmp as parachains_dmp, hrmp as parachains_hrmp, inclusion as parachains_inclusion, initializer as parachains_initializer, origin as parachains_origin, paras as parachains_paras, - paras_inherent as parachains_paras_inherent, runtime_api_impl::v2 as runtime_impl, + paras_inherent as parachains_paras_inherent, runtime_api_impl::v4 as runtime_impl, scheduler as parachains_scheduler, session_info as parachains_session_info, shared as parachains_shared, ump as parachains_ump, }; @@ -45,11 +45,12 @@ use pallet_session::historical as session_historical; use pallet_transaction_payment::{FeeDetails, RuntimeDispatchInfo}; use polkadot_runtime_parachains::reward_points::RewardValidatorsWithEraPoints; use primitives::{ - AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CommittedCandidateReceipt, - CoreState, GroupRotationInfo, Hash as HashT, Id as ParaId, InboundDownwardMessage, - InboundHrmpMessage, Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, - ScrapedOnChainVotes, SessionInfo as SessionInfoData, Signature, ValidationCode, - ValidationCodeHash, ValidatorId, ValidatorIndex, + AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, + Hash as HashT, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, + OccupiedCoreAssumption, PersistedValidationData, ScrapedOnChainVotes, + SessionInfo as SessionInfoData, Signature, ValidationCode, ValidationCodeHash, ValidatorId, + ValidatorIndex, }; use runtime_common::{ claims, impl_runtime_weights, paras_sudo_wrapper, BlockHashCount, BlockLength, @@ -546,7 +547,6 @@ pub mod pallet_test_notifier { use xcm::latest::prelude::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] pub struct Pallet(_); #[pallet::config] @@ -838,6 +838,10 @@ sp_api::impl_runtime_apis! { runtime_impl::session_info::(index) } + fn session_executor_params(session_index: SessionIndex) -> Option { + runtime_impl::session_executor_params::(session_index) + } + fn dmq_contents( recipient: ParaId, ) -> Vec> { @@ -874,6 +878,10 @@ sp_api::impl_runtime_apis! { { runtime_impl::validation_code_hash::(para_id, assumption) } + + fn disputes() -> Vec<(SessionIndex, CandidateHash, DisputeState)> { + runtime_impl::get_session_disputes::() + } } impl beefy_primitives::BeefyApi for Runtime { diff --git a/runtime/westend/src/lib.rs b/runtime/westend/src/lib.rs index d7274fa6f9f2..2c06753f7c30 100644 --- a/runtime/westend/src/lib.rs +++ b/runtime/westend/src/lib.rs @@ -36,8 +36,8 @@ use pallet_session::historical as session_historical; use pallet_transaction_payment::{CurrencyAdapter, FeeDetails, RuntimeDispatchInfo}; use parity_scale_codec::{Decode, Encode, MaxEncodedLen}; use primitives::{ - vstaging::ExecutorParams, AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, - CandidateHash, CommittedCandidateReceipt, CoreState, DisputeState, GroupRotationInfo, Hash, + AccountId, AccountIndex, Balance, BlockNumber, CandidateEvent, CandidateHash, + CommittedCandidateReceipt, CoreState, DisputeState, ExecutorParams, GroupRotationInfo, Hash, Id as ParaId, InboundDownwardMessage, InboundHrmpMessage, Moment, Nonce, OccupiedCoreAssumption, PersistedValidationData, PvfCheckStatement, ScrapedOnChainVotes, SessionInfo, Signature, ValidationCode, ValidationCodeHash, ValidatorId, ValidatorIndex, @@ -50,15 +50,12 @@ use runtime_common::{ }; use runtime_parachains::{ configuration as parachains_configuration, disputes as parachains_disputes, - disputes::slashing as parachains_slashing, - dmp as parachains_dmp, hrmp as parachains_hrmp, inclusion as parachains_inclusion, - initializer as parachains_initializer, origin as parachains_origin, paras as parachains_paras, + disputes::slashing as parachains_slashing, dmp as parachains_dmp, hrmp as parachains_hrmp, + inclusion as parachains_inclusion, initializer as parachains_initializer, + origin as parachains_origin, paras as parachains_paras, paras_inherent as parachains_paras_inherent, reward_points as parachains_reward_points, - runtime_api_impl::{ - v2 as parachains_runtime_api_impl, vstaging as parachains_runtime_api_impl_staging, - }, - scheduler as parachains_scheduler, session_info as parachains_session_info, - shared as parachains_shared, ump as parachains_ump, + runtime_api_impl::v4 as parachains_runtime_api_impl, scheduler as parachains_scheduler, + session_info as parachains_session_info, shared as parachains_shared, ump as parachains_ump, }; use scale_info::TypeInfo; use sp_core::{OpaqueMetadata, RuntimeDebug}; @@ -113,13 +110,13 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { spec_name: create_runtime_str!("westend"), impl_name: create_runtime_str!("parity-westend"), authoring_version: 2, - spec_version: 9370, + spec_version: 9390, impl_version: 0, #[cfg(not(feature = "disable-runtime-api"))] apis: RUNTIME_API_VERSIONS, #[cfg(feature = "disable-runtime-api")] apis: sp_version::create_apis_vec![[]], - transaction_version: 18, + transaction_version: 19, state_version: 1, }; @@ -391,6 +388,7 @@ impl pallet_election_provider_multi_phase::MinerConfig for Runtime { as frame_election_provider_support::ElectionDataProvider >::MaxVotesPerVoter; + type MaxWinners = MaxActiveValidators; // The unsigned submissions have to respect the weight of the submit_unsigned call, thus their // weight estimate function is wired to this call's weight. @@ -1196,17 +1194,23 @@ pub type SignedExtra = ( pallet_transaction_payment::ChargeTransactionPayment, ); -pub struct StakingMigrationV11OldPallet; -impl Get<&'static str> for StakingMigrationV11OldPallet { - fn get() -> &'static str { - "VoterList" +pub struct NominationPoolsMigrationV4OldPallet; +impl Get for NominationPoolsMigrationV4OldPallet { + fn get() -> Perbill { + Perbill::from_percent(100) } } /// All migrations that will run on the next runtime upgrade. /// /// Should be cleared after every release. -pub type Migrations = (clean_state_migration::CleanMigrate,); +pub type Migrations = ( + clean_state_migration::CleanMigrate, + pallet_nomination_pools::migration::v4::MigrateToV4< + Runtime, + NominationPoolsMigrationV4OldPallet, + >, +); /// Unchecked extrinsic type as expected by this runtime. pub type UncheckedExtrinsic = @@ -1334,7 +1338,6 @@ sp_api::impl_runtime_apis! { } } - #[api_version(4)] impl primitives::runtime_api::ParachainHost for Runtime { fn validators() -> Vec { parachains_runtime_api_impl::validators::() @@ -1399,7 +1402,7 @@ sp_api::impl_runtime_apis! { } fn session_executor_params(session_index: SessionIndex) -> Option { - parachains_runtime_api_impl_staging::session_executor_params::(session_index) + parachains_runtime_api_impl::session_executor_params::(session_index) } fn dmq_contents(recipient: ParaId) -> Vec> { @@ -1438,7 +1441,7 @@ sp_api::impl_runtime_apis! { } fn disputes() -> Vec<(SessionIndex, CandidateHash, DisputeState)> { - runtime_parachains::runtime_api_impl::vstaging::get_session_disputes::() + parachains_runtime_api_impl::get_session_disputes::() } } @@ -1859,7 +1862,7 @@ sp_api::impl_runtime_apis! { #[cfg(all(test, feature = "try-runtime"))] mod remote_tests { use super::*; - use frame_try_runtime::{runtime_decl_for_TryRuntime::TryRuntime, UpgradeCheckSelect}; + use frame_try_runtime::{runtime_decl_for_try_runtime::TryRuntime, UpgradeCheckSelect}; use remote_externalities::{ Builder, Mode, OfflineConfig, OnlineConfig, SnapshotConfig, Transport, }; diff --git a/runtime/westend/src/weights/pallet_nomination_pools.rs b/runtime/westend/src/weights/pallet_nomination_pools.rs index 6aec10617569..5fce09838824 100644 --- a/runtime/westend/src/weights/pallet_nomination_pools.rs +++ b/runtime/westend/src/weights/pallet_nomination_pools.rs @@ -16,22 +16,24 @@ //! Autogenerated weights for `pallet_nomination_pools` //! //! THIS FILE WAS AUTO-GENERATED USING THE SUBSTRATE BENCHMARK CLI VERSION 4.0.0-dev -//! DATE: 2023-02-27, STEPS: `50`, REPEAT: `20`, LOW RANGE: `[]`, HIGH RANGE: `[]` +//! DATE: 2023-03-06, STEPS: `50`, REPEAT: `20`, LOW RANGE: `[]`, HIGH RANGE: `[]` //! WORST CASE MAP SIZE: `1000000` -//! HOSTNAME: `bm6`, CPU: `Intel(R) Core(TM) i7-7700K CPU @ 4.20GHz` +//! HOSTNAME: `bm3`, CPU: `Intel(R) Core(TM) i7-7700K CPU @ 4.20GHz` //! EXECUTION: Some(Wasm), WASM-EXECUTION: Compiled, CHAIN: Some("westend-dev"), DB CACHE: 1024 // Executed Command: -// ./target/production/polkadot +// target/production/polkadot // benchmark // pallet -// --chain=westend-dev // --steps=50 // --repeat=20 -// --pallet=pallet_nomination_pools // --extrinsic=* // --execution=wasm // --wasm-execution=compiled +// --heap-pages=4096 +// --json-file=/var/lib/gitlab-runner/builds/zyw4fam_/0/parity/mirrors/polkadot/.git/.artifacts/bench.json +// --pallet=pallet_nomination_pools +// --chain=westend-dev // --header=./file_header.txt // --output=./runtime/westend/src/weights/ @@ -50,13 +52,15 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:2 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: NominationPools MaxPoolMembersPerPool (r:1 w:0) @@ -73,20 +77,22 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn join() -> Weight { // Proof Size summary in bytes: - // Measured: `3586` - // Estimated: `38468` - // Minimum execution time: 154_304 nanoseconds. - Weight::from_parts(155_220_000, 0) - .saturating_add(Weight::from_parts(0, 38468)) - .saturating_add(T::DbWeight::get().reads(17)) + // Measured: `3663` + // Estimated: `52915` + // Minimum execution time: 152_798_000 picoseconds. + Weight::from_parts(154_136_000, 0) + .saturating_add(Weight::from_parts(0, 52915)) + .saturating_add(T::DbWeight::get().reads(18)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:3 w:2) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -101,12 +107,12 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn bond_extra_transfer() -> Weight { // Proof Size summary in bytes: - // Measured: `3628` - // Estimated: `39063` - // Minimum execution time: 150_523 nanoseconds. - Weight::from_parts(154_073_000, 0) - .saturating_add(Weight::from_parts(0, 39063)) - .saturating_add(T::DbWeight::get().reads(14)) + // Measured: `3705` + // Estimated: `49550` + // Minimum execution time: 149_896_000 picoseconds. + Weight::from_parts(151_049_000, 0) + .saturating_add(Weight::from_parts(0, 49550)) + .saturating_add(T::DbWeight::get().reads(15)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools ClaimPermissions (r:1 w:0) @@ -114,9 +120,11 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:3 w:3) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -131,12 +139,12 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList ListBags (max_values: None, max_size: Some(82), added: 2557, mode: MaxEncodedLen) fn bond_extra_other() -> Weight { // Proof Size summary in bytes: - // Measured: `3427` - // Estimated: `38950` - // Minimum execution time: 162_554 nanoseconds. - Weight::from_parts(164_041_000, 0) - .saturating_add(Weight::from_parts(0, 38950)) - .saturating_add(T::DbWeight::get().reads(14)) + // Measured: `3504` + // Estimated: `50427` + // Minimum execution time: 161_501_000 picoseconds. + Weight::from_parts(163_539_000, 0) + .saturating_add(Weight::from_parts(0, 50427)) + .saturating_add(T::DbWeight::get().reads(15)) .saturating_add(T::DbWeight::get().writes(12)) } /// Storage: NominationPools ClaimPermissions (r:1 w:0) @@ -144,31 +152,35 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:1 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) fn claim_payout() -> Weight { // Proof Size summary in bytes: - // Measured: `1220` - // Estimated: `13485` - // Minimum execution time: 58_946 nanoseconds. - Weight::from_parts(59_799_000, 0) - .saturating_add(Weight::from_parts(0, 13485)) - .saturating_add(T::DbWeight::get().reads(5)) + // Measured: `1297` + // Estimated: `20012` + // Minimum execution time: 58_717_000 picoseconds. + Weight::from_parts(59_543_000, 0) + .saturating_add(Weight::from_parts(0, 20012)) + .saturating_add(T::DbWeight::get().reads(6)) .saturating_add(T::DbWeight::get().writes(4)) } /// Storage: NominationPools PoolMembers (r:1 w:1) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: System Account (r:2 w:1) /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) /// Storage: Staking CurrentEra (r:1 w:0) @@ -189,16 +201,16 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: NominationPools CounterForSubPoolsStorage (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn unbond() -> Weight { // Proof Size summary in bytes: - // Measured: `3865` - // Estimated: `43738` - // Minimum execution time: 155_416 nanoseconds. - Weight::from_parts(157_093_000, 0) - .saturating_add(Weight::from_parts(0, 43738)) - .saturating_add(T::DbWeight::get().reads(18)) + // Measured: `3942` + // Estimated: `59175` + // Minimum execution time: 154_589_000 picoseconds. + Weight::from_parts(155_708_000, 0) + .saturating_add(Weight::from_parts(0, 59175)) + .saturating_add(T::DbWeight::get().reads(19)) .saturating_add(T::DbWeight::get().writes(13)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:1) @@ -210,13 +222,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `s` is `[0, 100]`. fn pool_withdraw_unbonded(s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `1706` - // Estimated: `13025` - // Minimum execution time: 52_621 nanoseconds. - Weight::from_parts(53_508_921, 0) - .saturating_add(Weight::from_parts(0, 13025)) - // Standard Error: 790 - .saturating_add(Weight::from_parts(15_066, 0).saturating_mul(s.into())) + // Measured: `1710` + // Estimated: `18031` + // Minimum execution time: 52_663_000 picoseconds. + Weight::from_parts(54_091_646, 0) + .saturating_add(Weight::from_parts(0, 18031)) + // Standard Error: 2_889 + .saturating_add(Weight::from_parts(7_400, 0).saturating_mul(s.into())) .saturating_add(T::DbWeight::get().reads(5)) .saturating_add(T::DbWeight::get().writes(2)) } @@ -225,7 +237,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Staking CurrentEra (r:1 w:0) /// Proof: Staking CurrentEra (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools SubPoolsStorage (r:1 w:1) /// Proof: NominationPools SubPoolsStorage (max_values: None, max_size: Some(261), added: 2736, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) @@ -243,13 +255,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `s` is `[0, 100]`. fn withdraw_unbonded_update(s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `2230` - // Estimated: `22055` - // Minimum execution time: 102_019 nanoseconds. - Weight::from_parts(104_029_094, 0) - .saturating_add(Weight::from_parts(0, 22055)) - // Standard Error: 1_344 - .saturating_add(Weight::from_parts(13_886, 0).saturating_mul(s.into())) + // Measured: `2234` + // Estimated: `31021` + // Minimum execution time: 101_113_000 picoseconds. + Weight::from_parts(103_367_891, 0) + .saturating_add(Weight::from_parts(0, 31021)) + // Standard Error: 1_536 + .saturating_add(Weight::from_parts(9_937, 0).saturating_mul(s.into())) .saturating_add(T::DbWeight::get().reads(9)) .saturating_add(T::DbWeight::get().writes(8)) } @@ -258,7 +270,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Staking CurrentEra (r:1 w:0) /// Proof: Staking CurrentEra (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools SubPoolsStorage (r:1 w:1) /// Proof: NominationPools SubPoolsStorage (max_values: None, max_size: Some(261), added: 2736, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:1) @@ -282,7 +294,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools CounterForReversePoolIdLookup (r:1 w:1) /// Proof: NominationPools CounterForReversePoolIdLookup (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: NominationPools CounterForRewardPools (r:1 w:1) /// Proof: NominationPools CounterForRewardPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools CounterForSubPoolsStorage (r:1 w:1) @@ -296,15 +308,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools ClaimPermissions (r:0 w:1) /// Proof: NominationPools ClaimPermissions (max_values: None, max_size: Some(41), added: 2516, mode: MaxEncodedLen) /// The range of component `s` is `[0, 100]`. - fn withdraw_unbonded_kill(s: u32, ) -> Weight { + fn withdraw_unbonded_kill(_s: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `2618` - // Estimated: `45099` - // Minimum execution time: 161_236 nanoseconds. - Weight::from_parts(164_070_568, 0) - .saturating_add(Weight::from_parts(0, 45099)) - // Standard Error: 2_621 - .saturating_add(Weight::from_parts(10_297, 0).saturating_mul(s.into())) + // Measured: `2622` + // Estimated: `64001` + // Minimum execution time: 158_497_000 picoseconds. + Weight::from_parts(162_554_414, 0) + .saturating_add(Weight::from_parts(0, 64001)) .saturating_add(T::DbWeight::get().reads(20)) .saturating_add(T::DbWeight::get().writes(18)) } @@ -339,7 +349,7 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: Balances Locks (r:1 w:1) /// Proof: Balances Locks (max_values: None, max_size: Some(1299), added: 3774, mode: MaxEncodedLen) /// Storage: NominationPools RewardPools (r:1 w:1) - /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(60), added: 2535, mode: MaxEncodedLen) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) /// Storage: NominationPools CounterForRewardPools (r:1 w:1) /// Proof: NominationPools CounterForRewardPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools ReversePoolIdLookup (r:1 w:1) @@ -347,21 +357,21 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Storage: NominationPools CounterForReversePoolIdLookup (r:1 w:1) /// Proof: NominationPools CounterForReversePoolIdLookup (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Payee (r:0 w:1) /// Proof: Staking Payee (max_values: None, max_size: Some(73), added: 2548, mode: MaxEncodedLen) fn create() -> Weight { // Proof Size summary in bytes: // Measured: `1254` - // Estimated: `32002` - // Minimum execution time: 142_981 nanoseconds. - Weight::from_parts(144_237_000, 0) - .saturating_add(Weight::from_parts(0, 32002)) + // Estimated: `51890` + // Minimum execution time: 140_839_000 picoseconds. + Weight::from_parts(143_493_000, 0) + .saturating_add(Weight::from_parts(0, 51890)) .saturating_add(T::DbWeight::get().reads(21)) .saturating_add(T::DbWeight::get().writes(15)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) @@ -387,36 +397,36 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `n` is `[1, 16]`. fn nominate(n: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `1839` - // Estimated: `21998 + n * (2520 ±0)` - // Minimum execution time: 65_217 nanoseconds. - Weight::from_parts(65_750_377, 0) - .saturating_add(Weight::from_parts(0, 21998)) - // Standard Error: 7_685 - .saturating_add(Weight::from_parts(1_336_128, 0).saturating_mul(n.into())) + // Measured: `1843` + // Estimated: `33934 + n * (2520 ±0)` + // Minimum execution time: 65_363_000 picoseconds. + Weight::from_parts(66_269_952, 0) + .saturating_add(Weight::from_parts(0, 33934)) + // Standard Error: 10_011 + .saturating_add(Weight::from_parts(1_294_057, 0).saturating_mul(n.into())) .saturating_add(T::DbWeight::get().reads(12)) .saturating_add(T::DbWeight::get().reads((1_u64).saturating_mul(n.into()))) .saturating_add(T::DbWeight::get().writes(5)) .saturating_add(Weight::from_parts(0, 2520).saturating_mul(n.into())) } /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) /// Proof: Staking Ledger (max_values: None, max_size: Some(1091), added: 3566, mode: MaxEncodedLen) fn set_state() -> Weight { // Proof Size summary in bytes: - // Measured: `1427` - // Estimated: `8752` - // Minimum execution time: 35_863 nanoseconds. - Weight::from_parts(36_287_000, 0) - .saturating_add(Weight::from_parts(0, 8752)) + // Measured: `1431` + // Estimated: `11778` + // Minimum execution time: 36_747_000 picoseconds. + Weight::from_parts(37_477_000, 0) + .saturating_add(Weight::from_parts(0, 11778)) .saturating_add(T::DbWeight::get().reads(3)) .saturating_add(T::DbWeight::get().writes(1)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: NominationPools Metadata (r:1 w:1) /// Proof: NominationPools Metadata (max_values: None, max_size: Some(270), added: 2745, mode: MaxEncodedLen) /// Storage: NominationPools CounterForMetadata (r:1 w:1) @@ -424,13 +434,13 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// The range of component `n` is `[1, 256]`. fn set_metadata(n: u32, ) -> Weight { // Proof Size summary in bytes: - // Measured: `525` - // Estimated: `5883` - // Minimum execution time: 13_860 nanoseconds. - Weight::from_parts(14_425_990, 0) - .saturating_add(Weight::from_parts(0, 5883)) - // Standard Error: 94 - .saturating_add(Weight::from_parts(1_103, 0).saturating_mul(n.into())) + // Measured: `529` + // Estimated: `8909` + // Minimum execution time: 14_658_000 picoseconds. + Weight::from_parts(15_258_809, 0) + .saturating_add(Weight::from_parts(0, 8909)) + // Standard Error: 850 + .saturating_add(Weight::from_parts(3_615, 0).saturating_mul(n.into())) .saturating_add(T::DbWeight::get().reads(3)) .saturating_add(T::DbWeight::get().writes(2)) } @@ -442,31 +452,33 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: NominationPools MaxPoolMembersPerPool (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools MinCreateBond (r:0 w:1) /// Proof: NominationPools MinCreateBond (max_values: Some(1), max_size: Some(16), added: 511, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:0 w:1) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) /// Storage: NominationPools MaxPools (r:0 w:1) /// Proof: NominationPools MaxPools (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn set_configs() -> Weight { // Proof Size summary in bytes: // Measured: `0` // Estimated: `0` - // Minimum execution time: 5_097 nanoseconds. - Weight::from_parts(5_385_000, 0) + // Minimum execution time: 6_314_000 picoseconds. + Weight::from_parts(6_573_000, 0) .saturating_add(Weight::from_parts(0, 0)) - .saturating_add(T::DbWeight::get().writes(5)) + .saturating_add(T::DbWeight::get().writes(6)) } /// Storage: NominationPools BondedPools (r:1 w:1) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) fn update_roles() -> Weight { // Proof Size summary in bytes: - // Measured: `525` - // Estimated: `2639` - // Minimum execution time: 19_168 nanoseconds. - Weight::from_parts(19_566_000, 0) - .saturating_add(Weight::from_parts(0, 2639)) + // Measured: `529` + // Estimated: `3685` + // Minimum execution time: 19_551_000 picoseconds. + Weight::from_parts(19_827_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) .saturating_add(T::DbWeight::get().reads(1)) .saturating_add(T::DbWeight::get().writes(1)) } /// Storage: NominationPools BondedPools (r:1 w:0) - /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(164), added: 2639, mode: MaxEncodedLen) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) /// Storage: Staking Bonded (r:1 w:0) /// Proof: Staking Bonded (max_values: None, max_size: Some(72), added: 2547, mode: MaxEncodedLen) /// Storage: Staking Ledger (r:1 w:0) @@ -485,14 +497,56 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo /// Proof: VoterList CounterForListNodes (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) fn chill() -> Weight { // Proof Size summary in bytes: - // Measured: `2066` - // Estimated: `20489` - // Minimum execution time: 64_030 nanoseconds. - Weight::from_parts(64_573_000, 0) - .saturating_add(Weight::from_parts(0, 20489)) + // Measured: `2070` + // Estimated: `29455` + // Minimum execution time: 63_836_000 picoseconds. + Weight::from_parts(64_440_000, 0) + .saturating_add(Weight::from_parts(0, 29455)) .saturating_add(T::DbWeight::get().reads(9)) .saturating_add(T::DbWeight::get().writes(5)) } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + /// Storage: NominationPools RewardPools (r:1 w:1) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) + /// Storage: System Account (r:1 w:0) + /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) + fn set_commission() -> Weight { + // Proof Size summary in bytes: + // Measured: `832` + // Estimated: `12324` + // Minimum execution time: 32_116_000 picoseconds. + Weight::from_parts(32_651_000, 0) + .saturating_add(Weight::from_parts(0, 12324)) + .saturating_add(T::DbWeight::get().reads(4)) + .saturating_add(T::DbWeight::get().writes(2)) + } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + fn set_commission_max() -> Weight { + // Proof Size summary in bytes: + // Measured: `569` + // Estimated: `3685` + // Minimum execution time: 18_582_000 picoseconds. + Weight::from_parts(19_029_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) + .saturating_add(T::DbWeight::get().reads(1)) + .saturating_add(T::DbWeight::get().writes(1)) + } + /// Storage: NominationPools BondedPools (r:1 w:1) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + fn set_commission_change_rate() -> Weight { + // Proof Size summary in bytes: + // Measured: `529` + // Estimated: `3685` + // Minimum execution time: 19_668_000 picoseconds. + Weight::from_parts(20_103_000, 0) + .saturating_add(Weight::from_parts(0, 3685)) + .saturating_add(T::DbWeight::get().reads(1)) + .saturating_add(T::DbWeight::get().writes(1)) + } /// Storage: NominationPools PoolMembers (r:1 w:0) /// Proof: NominationPools PoolMembers (max_values: None, max_size: Some(717), added: 3192, mode: MaxEncodedLen) /// Storage: NominationPools ClaimPermissions (r:1 w:1) @@ -500,11 +554,29 @@ impl pallet_nomination_pools::WeightInfo for WeightInfo fn set_claim_permission() -> Weight { // Proof Size summary in bytes: // Measured: `508` - // Estimated: `5708` - // Minimum execution time: 14_137 nanoseconds. - Weight::from_parts(14_483_000, 0) - .saturating_add(Weight::from_parts(0, 5708)) + // Estimated: `7688` + // Minimum execution time: 14_730_000 picoseconds. + Weight::from_parts(15_088_000, 0) + .saturating_add(Weight::from_parts(0, 7688)) .saturating_add(T::DbWeight::get().reads(2)) .saturating_add(T::DbWeight::get().writes(1)) } + /// Storage: NominationPools BondedPools (r:1 w:0) + /// Proof: NominationPools BondedPools (max_values: None, max_size: Some(220), added: 2695, mode: MaxEncodedLen) + /// Storage: NominationPools RewardPools (r:1 w:1) + /// Proof: NominationPools RewardPools (max_values: None, max_size: Some(92), added: 2567, mode: MaxEncodedLen) + /// Storage: NominationPools GlobalMaxCommission (r:1 w:0) + /// Proof: NominationPools GlobalMaxCommission (max_values: Some(1), max_size: Some(4), added: 499, mode: MaxEncodedLen) + /// Storage: System Account (r:1 w:1) + /// Proof: System Account (max_values: None, max_size: Some(128), added: 2603, mode: MaxEncodedLen) + fn claim_commission() -> Weight { + // Proof Size summary in bytes: + // Measured: `1062` + // Estimated: `12324` + // Minimum execution time: 47_120_000 picoseconds. + Weight::from_parts(47_597_000, 0) + .saturating_add(Weight::from_parts(0, 12324)) + .saturating_add(T::DbWeight::get().reads(4)) + .saturating_add(T::DbWeight::get().writes(2)) + } } diff --git a/scripts/ci/gitlab/pipeline/test.yml b/scripts/ci/gitlab/pipeline/test.yml index 3c94fe31b46f..31224a6d1cad 100644 --- a/scripts/ci/gitlab/pipeline/test.yml +++ b/scripts/ci/gitlab/pipeline/test.yml @@ -33,8 +33,7 @@ test-linux-stable: - .common-refs - .pipeline-stopper-artifacts before_script: - - rustup show - - cargo --version + - !reference [.compiler-info, before_script] - !reference [.pipeline-stopper-vars, before_script] variables: RUST_TOOLCHAIN: stable diff --git a/utils/staking-miner/Cargo.lock b/utils/staking-miner/Cargo.lock deleted file mode 100644 index b966feddb58f..000000000000 --- a/utils/staking-miner/Cargo.lock +++ /dev/null @@ -1,4484 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "Inflector" -version = "0.11.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe438c63458706e03479442743baae6c88256498e6431708f6dfc520a26515d3" -dependencies = [ - "lazy_static", - "regex", -] - -[[package]] -name = "addr2line" -version = "0.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9ecd88a8c8378ca913a680cd98f0f13ac67383d35993f86c90a70e3f137816b" -dependencies = [ - "gimli", -] - -[[package]] -name = "adler" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" - -[[package]] -name = "ahash" -version = "0.7.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" -dependencies = [ - "getrandom 0.2.6", - "once_cell", - "version_check", -] - -[[package]] -name = "aho-corasick" -version = "0.7.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e37cfd5e7657ada45f742d6e99ca5788580b5c529dc78faf11ece6dc702656f" -dependencies = [ - "memchr", -] - -[[package]] -name = "ansi_term" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d52a9bb7ec0cf484c551830a7ce27bd20d67eac647e1befb56b0be4ee39a55d2" -dependencies = [ - "winapi", -] - -[[package]] -name = "anyhow" -version = "1.0.57" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f9b8508dccb7687a1d6c4ce66b2b0ecef467c94667de27d8d7fe1f8d2a9cdc" - -[[package]] -name = "approx" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cab112f0a86d568ea0e627cc1d6be74a1e9cd55214684db5561995f6dad897c6" -dependencies = [ - "num-traits", -] - -[[package]] -name = "arrayref" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" - -[[package]] -name = "arrayvec" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd9fd44efafa8690358b7408d253adf110036b88f55672a933f01d616ad9b1b9" -dependencies = [ - "nodrop", -] - -[[package]] -name = "arrayvec" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23b62fc65de8e4e7f52534fb52b0f3ed04746ae267519eef2a83941e8085068b" - -[[package]] -name = "arrayvec" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" - -[[package]] -name = "assert_cmd" -version = "2.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93ae1ddd39efd67689deb1979d80bad3bf7f2b09c6e6117c8d1f2443b5e2f83e" -dependencies = [ - "bstr", - "doc-comment", - "predicates", - "predicates-core", - "predicates-tree", - "wait-timeout", -] - -[[package]] -name = "async-trait" -version = "0.1.53" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed6aa3524a2dfcf9fe180c51eae2b58738348d819517ceadf95789c51fff7600" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi", - "libc", - "winapi", -] - -[[package]] -name = "autocfg" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" - -[[package]] -name = "backtrace" -version = "0.3.65" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11a17d453482a265fd5f8479f2a3f405566e6ca627837aaddb85af8b1ab8ef61" -dependencies = [ - "addr2line", - "cc", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", -] - -[[package]] -name = "base16ct" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "349a06037c7bf932dd7e7d1f653678b2038b9ad46a74102f1fc7bd7872678cce" - -[[package]] -name = "base58" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6107fe1be6682a68940da878d9e9f5e90ca5745b3dec9fd1bb393c8777d4f581" - -[[package]] -name = "base64" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" - -[[package]] -name = "beef" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bed554bd50246729a1ec158d08aa3235d1b69d94ad120ebe187e28894787e736" -dependencies = [ - "serde", -] - -[[package]] -name = "bitflags" -version = "1.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" - -[[package]] -name = "bitvec" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1489fcb93a5bb47da0462ca93ad252ad6af2145cce58d10d46a83931ba9f016b" -dependencies = [ - "funty", - "radium", - "tap", - "wyz", -] - -[[package]] -name = "blake2" -version = "0.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9cf849ee05b2ee5fba5e36f97ff8ec2533916700fc0758d40d92136a42f3388" -dependencies = [ - "digest 0.10.3", -] - -[[package]] -name = "blake2-rfc" -version = "0.2.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d6d530bdd2d52966a6d03b7a964add7ae1a288d25214066fd4b600f0f796400" -dependencies = [ - "arrayvec 0.4.12", - "constant_time_eq", -] - -[[package]] -name = "block-buffer" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0940dc441f31689269e10ac70eb1002a3a1d3ad1390e030043662eb7fe4688b" -dependencies = [ - "block-padding", - "byte-tools", - "byteorder", - "generic-array 0.12.4", -] - -[[package]] -name = "block-buffer" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4152116fd6e9dadb291ae18fc1ec3575ed6d84c29642d97890f4b4a3417297e4" -dependencies = [ - "generic-array 0.14.5", -] - -[[package]] -name = "block-buffer" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bf7fe51849ea569fd452f37822f606a5cabb684dc918707a0193fd4664ff324" -dependencies = [ - "generic-array 0.14.5", -] - -[[package]] -name = "block-padding" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa79dedbb091f449f1f39e53edf88d5dbe95f895dae6135a8d7b881fb5af73f5" -dependencies = [ - "byte-tools", -] - -[[package]] -name = "bstr" -version = "0.2.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba3569f383e8f1598449f1a423e72e99569137b47740b1da11ef19af3d5c3223" -dependencies = [ - "lazy_static", - "memchr", - "regex-automata", -] - -[[package]] -name = "bumpalo" -version = "3.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a45a46ab1f2412e53d3a0ade76ffad2025804294569aae387231a0cd6e0899" - -[[package]] -name = "byte-slice-cast" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87c5fdd0166095e1d463fc6cc01aa8ce547ad77a4e84d42eb6762b084e28067e" - -[[package]] -name = "byte-tools" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3b5ca7a04898ad4bcd41c90c5285445ff5b791899bb1b0abdd2a2aa791211d7" - -[[package]] -name = "byteorder" -version = "1.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" - -[[package]] -name = "bytes" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" - -[[package]] -name = "cc" -version = "1.0.73" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "chrono" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "670ad68c9088c2a963aaa298cb369688cf3f9465ce5e2d4ca10e6e0098a1ce73" -dependencies = [ - "libc", - "num-integer", - "num-traits", - "winapi", -] - -[[package]] -name = "clap" -version = "3.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85a35a599b11c089a7f49105658d089b8f2cf0882993c17daf6de15285c2c35d" -dependencies = [ - "atty", - "bitflags", - "clap_derive", - "clap_lex", - "indexmap", - "lazy_static", - "strsim", - "termcolor", - "textwrap", -] - -[[package]] -name = "clap_derive" -version = "3.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3aab4734e083b809aaf5794e14e756d1c798d2c69c7f7de7a09a2f5214993c1" -dependencies = [ - "heck 0.4.0", - "proc-macro-error", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "clap_lex" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a37c35f1112dad5e6e0b1adaff798507497a18fceeb30cceb3bae7d1427b9213" -dependencies = [ - "os_str_bytes", -] - -[[package]] -name = "const-oid" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4c78c047431fee22c1a7bb92e00ad095a02a983affe4d8a72e2a2c62c1b94f3" - -[[package]] -name = "constant_time_eq" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "245097e9a4535ee1e3e3931fcfcd55a796a44c643e8596ff6566d68f09b87bbc" - -[[package]] -name = "core-foundation" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "194a7a9e6de53fa55116934067c844d9d749312f75c6f6d0980e8c252f8c2146" -dependencies = [ - "core-foundation-sys", - "libc", -] - -[[package]] -name = "core-foundation-sys" -version = "0.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" - -[[package]] -name = "cpufeatures" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59a6001667ab124aebae2a495118e11d30984c3a653e99d86d58971708cf5e4b" -dependencies = [ - "libc", -] - -[[package]] -name = "crunchy" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" - -[[package]] -name = "crypto-bigint" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c6a1d5fa1de37e071642dfa44ec552ca5b299adb128fab16138e24b548fd21" -dependencies = [ - "generic-array 0.14.5", - "rand_core 0.6.3", - "subtle", - "zeroize", -] - -[[package]] -name = "crypto-common" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57952ca27b5e3606ff4dd79b0020231aaf9d6aa76dc05fd30137538c50bd3ce8" -dependencies = [ - "generic-array 0.14.5", - "typenum", -] - -[[package]] -name = "crypto-mac" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b584a330336237c1eecd3e94266efb216c56ed91225d634cb2991c5f3fd1aeab" -dependencies = [ - "generic-array 0.14.5", - "subtle", -] - -[[package]] -name = "crypto-mac" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1d1a86f49236c215f271d40892d5fc950490551400b02ef360692c29815c714" -dependencies = [ - "generic-array 0.14.5", - "subtle", -] - -[[package]] -name = "curve25519-dalek" -version = "2.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a9b85542f99a2dfa2a1b8e192662741c9859a846b296bef1c92ef9b58b5a216" -dependencies = [ - "byteorder", - "digest 0.8.1", - "rand_core 0.5.1", - "subtle", - "zeroize", -] - -[[package]] -name = "curve25519-dalek" -version = "3.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b9fdf9972b2bd6af2d913799d9ebc165ea4d2e65878e329d9c6b372c4491b61" -dependencies = [ - "byteorder", - "digest 0.9.0", - "rand_core 0.5.1", - "subtle", - "zeroize", -] - -[[package]] -name = "darling" -version = "0.14.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4529658bdda7fd6769b8614be250cdcfc3aeb0ee72fe66f9e41e5e5eb73eac02" -dependencies = [ - "darling_core", - "darling_macro", -] - -[[package]] -name = "darling_core" -version = "0.14.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "649c91bc01e8b1eac09fb91e8dbc7d517684ca6be8ebc75bb9cafc894f9fdb6f" -dependencies = [ - "fnv", - "ident_case", - "proc-macro2", - "quote", - "strsim", - "syn", -] - -[[package]] -name = "darling_macro" -version = "0.14.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddfc69c5bfcbd2fc09a0f38451d2daf0e372e367986a83906d1b0dbc88134fb5" -dependencies = [ - "darling_core", - "quote", - "syn", -] - -[[package]] -name = "der" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6919815d73839e7ad218de758883aae3a257ba6759ce7a9992501efbb53d705c" -dependencies = [ - "const-oid", -] - -[[package]] -name = "derivative" -version = "2.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "derive_more" -version = "0.99.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fb810d30a7c1953f91334de7244731fc3f3c10d7fe163338a35b9f640960321" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "difflib" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6184e33543162437515c2e2b48714794e37845ec9851711914eec9d308f6ebe8" - -[[package]] -name = "digest" -version = "0.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3d0c8c8752312f9713efd397ff63acb9f85585afbf179282e720e7704954dd5" -dependencies = [ - "generic-array 0.12.4", -] - -[[package]] -name = "digest" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3dd60d1080a57a05ab032377049e0591415d2b31afd7028356dbf3cc6dcb066" -dependencies = [ - "generic-array 0.14.5", -] - -[[package]] -name = "digest" -version = "0.10.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2fb860ca6fafa5552fb6d0e816a69c8e49f0908bf524e30a90d97c85892d506" -dependencies = [ - "block-buffer 0.10.2", - "crypto-common", - "subtle", -] - -[[package]] -name = "doc-comment" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" - -[[package]] -name = "downcast-rs" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ea835d29036a4087793836fa931b08837ad5e957da9e23886b29586fb9b6650" - -[[package]] -name = "dyn-clonable" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e9232f0e607a262ceb9bd5141a3dfb3e4db6994b31989bbfd845878cba59fd4" -dependencies = [ - "dyn-clonable-impl", - "dyn-clone", -] - -[[package]] -name = "dyn-clonable-impl" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "558e40ea573c374cf53507fd240b7ee2f5477df7cfebdb97323ec61c719399c5" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "dyn-clone" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21e50f3adc76d6a43f5ed73b698a87d0760ca74617f60f7c3b879003536fdd28" - -[[package]] -name = "ecdsa" -version = "0.13.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0d69ae62e0ce582d56380743515fefaf1a8c70cec685d9677636d7e30ae9dc9" -dependencies = [ - "der", - "elliptic-curve", - "rfc6979", - "signature", -] - -[[package]] -name = "ed25519" -version = "1.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d5c4b5e5959dc2c2b89918d8e2cc40fcdd623cef026ed09d2f0ee05199dc8e4" -dependencies = [ - "signature", -] - -[[package]] -name = "ed25519-dalek" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c762bae6dcaf24c4c84667b8579785430908723d5c889f469d76a41d59cc7a9d" -dependencies = [ - "curve25519-dalek 3.2.0", - "ed25519", - "rand 0.7.3", - "serde", - "sha2 0.9.9", - "zeroize", -] - -[[package]] -name = "either" -version = "1.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" - -[[package]] -name = "elliptic-curve" -version = "0.11.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25b477563c2bfed38a3b7a60964c49e058b2510ad3f12ba3483fd8f62c2306d6" -dependencies = [ - "base16ct", - "crypto-bigint", - "der", - "ff", - "generic-array 0.14.5", - "group", - "rand_core 0.6.3", - "sec1", - "subtle", - "zeroize", -] - -[[package]] -name = "env_logger" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b2cf0344971ee6c64c31be0d530793fba457d322dfec2810c453d0ef228f9c3" -dependencies = [ - "atty", - "humantime", - "log", - "regex", - "termcolor", -] - -[[package]] -name = "environmental" -version = "1.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68b91989ae21441195d7d9b9993a2f9295c7e1a8c96255d8b729accddc124797" - -[[package]] -name = "fake-simd" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88a8acf291dafb59c2d96e8f59828f3838bb1a70398823ade51a84de6a6deed" - -[[package]] -name = "ff" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2958d04124b9f27f175eaeb9a9f383d026098aa837eadd8ba22c11f13a05b9e" -dependencies = [ - "rand_core 0.6.3", - "subtle", -] - -[[package]] -name = "fixed-hash" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfcf0ed7fe52a17a03854ec54a9f76d6d84508d1c0e66bc1793301c73fc8493c" -dependencies = [ - "byteorder", - "rand 0.8.5", - "rustc-hex", - "static_assertions", -] - -[[package]] -name = "fnv" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" - -[[package]] -name = "frame-benchmarking" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "frame-support", - "frame-system", - "linregress", - "log", - "parity-scale-codec", - "paste", - "scale-info", - "serde", - "sp-api", - "sp-application-crypto 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "frame-election-provider-solution-type" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "frame-election-provider-support" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "frame-election-provider-solution-type", - "frame-support", - "frame-system", - "parity-scale-codec", - "scale-info", - "sp-arithmetic 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-npos-elections", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "frame-metadata" -version = "15.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df6bb8542ef006ef0de09a5c4420787d79823c0ed7924225822362fd2bf2ff2d" -dependencies = [ - "cfg-if", - "parity-scale-codec", - "scale-info", - "serde", -] - -[[package]] -name = "frame-support" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "bitflags", - "frame-metadata", - "frame-support-procedural", - "impl-trait-for-tuples", - "k256", - "log", - "once_cell", - "parity-scale-codec", - "paste", - "scale-info", - "serde", - "smallvec", - "sp-arithmetic 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-core-hashing-proc-macro", - "sp-inherents", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-staking", - "sp-state-machine 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-tracing 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "tt-call", -] - -[[package]] -name = "frame-support-procedural" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "Inflector", - "frame-support-procedural-tools", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "frame-support-procedural-tools" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "frame-support-procedural-tools-derive", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "frame-support-procedural-tools-derive" -version = "3.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "frame-system" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "frame-support", - "log", - "parity-scale-codec", - "scale-info", - "serde", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-version", -] - -[[package]] -name = "funty" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" - -[[package]] -name = "futures" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f73fe65f54d1e12b726f517d3e2135ca3125a437b6d998caf1962961f7172d9e" -dependencies = [ - "futures-channel", - "futures-core", - "futures-executor", - "futures-io", - "futures-sink", - "futures-task", - "futures-util", -] - -[[package]] -name = "futures-channel" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3083ce4b914124575708913bca19bfe887522d6e2e6d0952943f5eac4a74010" -dependencies = [ - "futures-core", - "futures-sink", -] - -[[package]] -name = "futures-core" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c09fd04b7e4073ac7156a9539b57a484a8ea920f79c7c675d05d289ab6110d3" - -[[package]] -name = "futures-executor" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9420b90cfa29e327d0429f19be13e7ddb68fa1cccb09d65e5706b8c7a749b8a6" -dependencies = [ - "futures-core", - "futures-task", - "futures-util", - "num_cpus", -] - -[[package]] -name = "futures-io" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc4045962a5a5e935ee2fdedaa4e08284547402885ab326734432bed5d12966b" - -[[package]] -name = "futures-macro" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33c1e13800337f4d4d7a316bf45a567dbcb6ffe087f16424852d97e97a91f512" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "futures-sink" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21163e139fa306126e6eedaf49ecdb4588f939600f0b1e770f4205ee4b7fa868" - -[[package]] -name = "futures-task" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57c66a976bf5909d801bbef33416c41372779507e7a6b3a5e25e4749c58f776a" - -[[package]] -name = "futures-timer" -version = "3.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" - -[[package]] -name = "futures-util" -version = "0.3.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8b7abd5d659d9b90c8cba917f6ec750a74e2dc23902ef9cd4cc8c8b22e6036a" -dependencies = [ - "futures-channel", - "futures-core", - "futures-io", - "futures-macro", - "futures-sink", - "futures-task", - "memchr", - "pin-project-lite", - "pin-utils", - "slab", -] - -[[package]] -name = "generic-array" -version = "0.12.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffdf9f34f1447443d37393cc6c2b8313aebddcd96906caf34e54c68d8e57d7bd" -dependencies = [ - "typenum", -] - -[[package]] -name = "generic-array" -version = "0.14.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd48d33ec7f05fbfa152300fdad764757cbded343c1aa1cff2fbaf4134851803" -dependencies = [ - "typenum", - "version_check", -] - -[[package]] -name = "getrandom" -version = "0.1.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fc3cb4d91f53b50155bdcfd23f6a4c39ae1969c2ae85982b135750cccaf5fce" -dependencies = [ - "cfg-if", - "js-sys", - "libc", - "wasi 0.9.0+wasi-snapshot-preview1", - "wasm-bindgen", -] - -[[package]] -name = "getrandom" -version = "0.2.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9be70c98951c83b8d2f8f60d7065fa6d5146873094452a1008da8c2f1e4205ad" -dependencies = [ - "cfg-if", - "libc", - "wasi 0.10.2+wasi-snapshot-preview1", -] - -[[package]] -name = "gimli" -version = "0.26.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78cc372d058dcf6d5ecd98510e7fbc9e5aec4d21de70f65fea8fecebcd881bd4" - -[[package]] -name = "group" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc5ac374b108929de78460075f3dc439fa66df9d8fc77e8f12caa5165fcf0c89" -dependencies = [ - "ff", - "rand_core 0.6.3", - "subtle", -] - -[[package]] -name = "hash-db" -version = "0.15.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d23bd4e7b5eda0d0f3a307e8b381fdc8ba9000f26fbe912250c0a4cc3956364a" - -[[package]] -name = "hash256-std-hasher" -version = "0.15.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92c171d55b98633f4ed3860808f004099b36c1cc29c42cfc53aa8591b21efcf2" -dependencies = [ - "crunchy", -] - -[[package]] -name = "hashbrown" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab5ef0d4909ef3724cc8cce6ccc8572c5c817592e9285f5464f8e86f8bd3726e" -dependencies = [ - "ahash", -] - -[[package]] -name = "hashbrown" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db0d4cf898abf0081f964436dc980e96670a0f36863e4b83aaacdb65c9d7ccc3" -dependencies = [ - "ahash", -] - -[[package]] -name = "heck" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" -dependencies = [ - "unicode-segmentation", -] - -[[package]] -name = "heck" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2540771e65fc8cb83cd6e8a237f70c319bd5c29f78ed1084ba5d50eeac86f7f9" - -[[package]] -name = "hermit-abi" -version = "0.1.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - -[[package]] -name = "hex" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" - -[[package]] -name = "hmac" -version = "0.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "126888268dcc288495a26bf004b38c5fdbb31682f992c84ceb046a1f0fe38840" -dependencies = [ - "crypto-mac 0.8.0", - "digest 0.9.0", -] - -[[package]] -name = "hmac" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a2a2320eb7ec0ebe8da8f744d7812d9fc4cb4d09344ac01898dbcb6a20ae69b" -dependencies = [ - "crypto-mac 0.11.1", - "digest 0.9.0", -] - -[[package]] -name = "hmac-drbg" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17ea0a1394df5b6574da6e0c1ade9e78868c9fb0a4e5ef4428e32da4676b85b1" -dependencies = [ - "digest 0.9.0", - "generic-array 0.14.5", - "hmac 0.8.1", -] - -[[package]] -name = "http" -version = "0.2.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff8670570af52249509a86f5e3e18a08c60b177071826898fde8997cf5f6bfbb" -dependencies = [ - "bytes", - "fnv", - "itoa 1.0.1", -] - -[[package]] -name = "http-body" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" -dependencies = [ - "bytes", - "http", - "pin-project-lite", -] - -[[package]] -name = "httparse" -version = "1.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "496ce29bb5a52785b44e0f7ca2847ae0bb839c9bd28f69acac9b99d461c0c04c" - -[[package]] -name = "httpdate" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a1e36c821dbe04574f602848a19f742f4fb3c98d40449f11bcad18d6b17421" - -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" - -[[package]] -name = "hyper" -version = "0.14.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b26ae0a80afebe130861d90abf98e3814a4f28a4c6ffeb5ab8ebb2be311e0ef2" -dependencies = [ - "bytes", - "futures-channel", - "futures-core", - "futures-util", - "http", - "http-body", - "httparse", - "httpdate", - "itoa 1.0.1", - "pin-project-lite", - "tokio", - "tower-service", - "tracing", - "want", -] - -[[package]] -name = "ident_case" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" - -[[package]] -name = "impl-codec" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba6a270039626615617f3f36d15fc827041df3b78c439da2cadfa47455a77f2f" -dependencies = [ - "parity-scale-codec", -] - -[[package]] -name = "impl-serde" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4551f042f3438e64dbd6226b20527fc84a6e1fe65688b58746a2f53623f25f5c" -dependencies = [ - "serde", -] - -[[package]] -name = "impl-trait-for-tuples" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11d7a9f6330b71fea57921c9b61c47ee6e84f72d394754eff6163ae67e7395eb" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "indexmap" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f647032dfaa1f8b6dc29bd3edb7bbef4861b8b8007ebb118d6db284fd59f6ee" -dependencies = [ - "autocfg", - "hashbrown 0.11.2", -] - -[[package]] -name = "integer-sqrt" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "276ec31bcb4a9ee45f58bec6f9ec700ae4cf4f4f8f2fa7e06cb406bd5ffdd770" -dependencies = [ - "num-traits", -] - -[[package]] -name = "itertools" -version = "0.10.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9a9d19fa1e79b6215ff29b9d6880b706147f16e9b1dbb1e4e5947b5b02bc5e3" -dependencies = [ - "either", -] - -[[package]] -name = "itoa" -version = "0.4.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b71991ff56294aa922b450139ee08b3bfc70982c6b2c7562771375cf73542dd4" - -[[package]] -name = "itoa" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1aab8fc367588b89dcee83ab0fd66b72b50b72fa1904d7095045ace2b0c81c35" - -[[package]] -name = "js-sys" -version = "0.3.57" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671a26f820db17c2a2750743f1dd03bafd15b98c9f30c7c2628c024c05d73397" -dependencies = [ - "wasm-bindgen", -] - -[[package]] -name = "jsonrpsee" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91dc760c341fa81173f9a434931aaf32baad5552b0230cc6c93e8fb7eaad4c19" -dependencies = [ - "jsonrpsee-client-transport", - "jsonrpsee-core", - "jsonrpsee-proc-macros", - "jsonrpsee-types", - "jsonrpsee-ws-client", -] - -[[package]] -name = "jsonrpsee-client-transport" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "765f7a36d5087f74e3b3b47805c2188fef8eb54afcb587b078d9f8ebfe9c7220" -dependencies = [ - "futures", - "http", - "jsonrpsee-core", - "jsonrpsee-types", - "pin-project", - "rustls-native-certs", - "soketto", - "thiserror", - "tokio", - "tokio-rustls", - "tokio-util", - "tracing", - "webpki-roots", -] - -[[package]] -name = "jsonrpsee-core" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82ef77ecd20c2254d54f5da8c0738eacca61e6b6511268a8f2753e3148c6c706" -dependencies = [ - "anyhow", - "arrayvec 0.7.2", - "async-trait", - "beef", - "futures-channel", - "futures-util", - "hyper", - "jsonrpsee-types", - "rustc-hash", - "serde", - "serde_json", - "soketto", - "thiserror", - "tokio", - "tracing", -] - -[[package]] -name = "jsonrpsee-proc-macros" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7291c72805bc7d413b457e50d8ef3e87aa554da65ecbbc278abb7dfc283e7f0" -dependencies = [ - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "jsonrpsee-types" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38b6aa52f322cbf20c762407629b8300f39bcc0cf0619840d9252a2f65fd2dd9" -dependencies = [ - "anyhow", - "beef", - "serde", - "serde_json", - "thiserror", - "tracing", -] - -[[package]] -name = "jsonrpsee-ws-client" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd66d18bab78d956df24dd0d2e41e4c00afbb818fda94a98264bdd12ce8506ac" -dependencies = [ - "jsonrpsee-client-transport", - "jsonrpsee-core", - "jsonrpsee-types", -] - -[[package]] -name = "k256" -version = "0.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19c3a5e0a0b8450278feda242592512e09f61c72e018b8cd5c859482802daf2d" -dependencies = [ - "cfg-if", - "ecdsa", - "elliptic-curve", - "sec1", -] - -[[package]] -name = "keccak" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67c21572b4949434e4fc1e1978b99c5f77064153c59d998bf13ecd96fb5ecba7" - -[[package]] -name = "kvdb" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a301d8ecb7989d4a6e2c57a49baca77d353bdbf879909debe3f375fe25d61f86" -dependencies = [ - "parity-util-mem", - "smallvec", -] - -[[package]] -name = "lazy_static" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" - -[[package]] -name = "libc" -version = "0.2.125" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5916d2ae698f6de9bfb891ad7a8d65c09d232dc58cc4ac433c7da3b2fd84bc2b" - -[[package]] -name = "libm" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33a33a362ce288760ec6a508b94caaec573ae7d3bbbd91b87aa0bad4456839db" - -[[package]] -name = "libsecp256k1" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0452aac8bab02242429380e9b2f94ea20cea2b37e2c1777a1358799bbe97f37" -dependencies = [ - "arrayref", - "base64", - "digest 0.9.0", - "hmac-drbg", - "libsecp256k1-core", - "libsecp256k1-gen-ecmult", - "libsecp256k1-gen-genmult", - "rand 0.8.5", - "serde", - "sha2 0.9.9", - "typenum", -] - -[[package]] -name = "libsecp256k1-core" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5be9b9bb642d8522a44d533eab56c16c738301965504753b03ad1de3425d5451" -dependencies = [ - "crunchy", - "digest 0.9.0", - "subtle", -] - -[[package]] -name = "libsecp256k1-gen-ecmult" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3038c808c55c87e8a172643a7d87187fc6c4174468159cb3090659d55bcb4809" -dependencies = [ - "libsecp256k1-core", -] - -[[package]] -name = "libsecp256k1-gen-genmult" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3db8d6ba2cec9eacc40e6e8ccc98931840301f1006e95647ceb2dd5c3aa06f7c" -dependencies = [ - "libsecp256k1-core", -] - -[[package]] -name = "linregress" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6c601a85f5ecd1aba625247bca0031585fb1c446461b142878a16f8245ddeb8" -dependencies = [ - "nalgebra", - "statrs", -] - -[[package]] -name = "lock_api" -version = "0.4.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327fa5b6a6940e4699ec49a9beae1ea4845c6bab9314e4f84ac68742139d8c53" -dependencies = [ - "autocfg", - "scopeguard", -] - -[[package]] -name = "log" -version = "0.4.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" -dependencies = [ - "cfg-if", -] - -[[package]] -name = "lru" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32613e41de4c47ab04970c348ca7ae7382cf116625755af070b008a15516a889" -dependencies = [ - "hashbrown 0.11.2", -] - -[[package]] -name = "matchers" -version = "0.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" -dependencies = [ - "regex-automata", -] - -[[package]] -name = "matchers" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" -dependencies = [ - "regex-automata", -] - -[[package]] -name = "matrixmultiply" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "add85d4dd35074e6fedc608f8c8f513a3548619a9024b751949ef0e8e45a4d84" -dependencies = [ - "rawpointer", -] - -[[package]] -name = "memchr" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" - -[[package]] -name = "memory-db" -version = "0.29.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6566c70c1016f525ced45d7b7f97730a2bafb037c788211d0c186ef5b2189f0a" -dependencies = [ - "hash-db", - "hashbrown 0.12.1", - "parity-util-mem", -] - -[[package]] -name = "memory_units" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71d96e3f3c0b6325d8ccd83c33b28acb183edcb6c67938ba104ec546854b0882" - -[[package]] -name = "merlin" -version = "2.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e261cf0f8b3c42ded9f7d2bb59dea03aa52bc8a1cbc7482f9fc3fd1229d3b42" -dependencies = [ - "byteorder", - "keccak", - "rand_core 0.5.1", - "zeroize", -] - -[[package]] -name = "miniz_oxide" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b29bd4bc3f33391105ebee3589c19197c4271e3e5a9ec9bfe8127eeff8f082" -dependencies = [ - "adler", -] - -[[package]] -name = "mio" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52da4364ffb0e4fe33a9841a98a3f3014fb964045ce4f7a45a398243c8d6b0c9" -dependencies = [ - "libc", - "log", - "miow", - "ntapi", - "wasi 0.11.0+wasi-snapshot-preview1", - "winapi", -] - -[[package]] -name = "miow" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f1c5b025cda876f66ef43a113f91ebc9f4ccef34843000e0adf6ebbab84e21" -dependencies = [ - "winapi", -] - -[[package]] -name = "nalgebra" -version = "0.27.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "462fffe4002f4f2e1f6a9dcf12cc1a6fc0e15989014efc02a941d3e0f5dc2120" -dependencies = [ - "approx", - "matrixmultiply", - "nalgebra-macros", - "num-complex", - "num-rational 0.4.0", - "num-traits", - "rand 0.8.5", - "rand_distr", - "simba", - "typenum", -] - -[[package]] -name = "nalgebra-macros" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01fcc0b8149b4632adc89ac3b7b31a12fb6099a0317a4eb2ebff574ef7de7218" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "nodrop" -version = "0.1.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb" - -[[package]] -name = "ntapi" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c28774a7fd2fbb4f0babd8237ce554b73af68021b5f695a3cebd6c59bac0980f" -dependencies = [ - "winapi", -] - -[[package]] -name = "num-bigint" -version = "0.2.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "090c7f9998ee0ff65aa5b723e4009f7b217707f1fb5ea551329cc4d6231fb304" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-complex" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fbc387afefefd5e9e39493299f3069e14a140dd34dc19b4c1c1a8fddb6a790" -dependencies = [ - "num-traits", -] - -[[package]] -name = "num-format" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bafe4179722c2894288ee77a9f044f02811c86af699344c498b0840c698a2465" -dependencies = [ - "arrayvec 0.4.12", - "itoa 0.4.8", -] - -[[package]] -name = "num-integer" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" -dependencies = [ - "autocfg", - "num-traits", -] - -[[package]] -name = "num-rational" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c000134b5dbf44adc5cb772486d335293351644b801551abe8f75c84cfa4aef" -dependencies = [ - "autocfg", - "num-bigint", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-rational" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d41702bd167c2df5520b384281bc111a4b5efcf7fbc4c9c222c815b07e0a6a6a" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-traits" -version = "0.2.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "578ede34cf02f8924ab9447f50c28075b4d3e5b269972345e7e0372b38c6cdcd" -dependencies = [ - "autocfg", - "libm", -] - -[[package]] -name = "num_cpus" -version = "1.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19e64526ebdee182341572e50e9ad03965aa510cd94427a4549448f285e957a1" -dependencies = [ - "hermit-abi", - "libc", -] - -[[package]] -name = "object" -version = "0.28.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40bec70ba014595f99f7aa110b84331ffe1ee9aece7fe6f387cc7e3ecda4d456" -dependencies = [ - "memchr", -] - -[[package]] -name = "once_cell" -version = "1.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f3e037eac156d1775da914196f0f37741a274155e34a0b7e427c35d2a2ecb9" - -[[package]] -name = "opaque-debug" -version = "0.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2839e79665f131bdb5782e51f2c6c9599c133c6098982a54c794358bf432529c" - -[[package]] -name = "opaque-debug" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" - -[[package]] -name = "openssl-probe" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" - -[[package]] -name = "os_str_bytes" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e22443d1643a904602595ba1cd8f7d896afe56d26712531c5ff73a15b2fbf64" - -[[package]] -name = "pallet-election-provider-multi-phase" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "frame-benchmarking", - "frame-election-provider-support", - "frame-support", - "frame-system", - "log", - "parity-scale-codec", - "rand 0.7.3", - "scale-info", - "sp-arithmetic 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-npos-elections", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "static_assertions", - "strum", -] - -[[package]] -name = "pallet-transaction-payment" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "frame-support", - "frame-system", - "parity-scale-codec", - "scale-info", - "serde", - "smallvec", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "parity-scale-codec" -version = "3.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8b44461635bbb1a0300f100a841e571e7d919c81c73075ef5d152ffdb521066" -dependencies = [ - "arrayvec 0.7.2", - "bitvec", - "byte-slice-cast", - "impl-trait-for-tuples", - "parity-scale-codec-derive", - "serde", -] - -[[package]] -name = "parity-scale-codec-derive" -version = "3.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c45ed1f39709f5a89338fab50e59816b2e8815f5bb58276e7ddf9afd495f73f8" -dependencies = [ - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "parity-util-mem" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c32561d248d352148124f036cac253a644685a21dc9fea383eb4907d7bd35a8f" -dependencies = [ - "cfg-if", - "hashbrown 0.12.1", - "impl-trait-for-tuples", - "parity-util-mem-derive", - "parking_lot", - "primitive-types", - "winapi", -] - -[[package]] -name = "parity-util-mem-derive" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f557c32c6d268a07c921471619c0295f5efad3a0e76d4f97a05c091a51d110b2" -dependencies = [ - "proc-macro2", - "syn", - "synstructure", -] - -[[package]] -name = "parity-wasm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be5e13c266502aadf83426d87d81a0f5d1ef45b8027f5a471c360abfe4bfae92" - -[[package]] -name = "parking_lot" -version = "0.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f5ec2493a61ac0506c0f4199f99070cbe83857b0337006a30f3e6719b8ef58" -dependencies = [ - "lock_api", - "parking_lot_core", -] - -[[package]] -name = "parking_lot_core" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09a279cbf25cb0757810394fbc1e359949b59e348145c643a939a525692e6929" -dependencies = [ - "cfg-if", - "libc", - "redox_syscall", - "smallvec", - "windows-sys", -] - -[[package]] -name = "paste" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c520e05135d6e763148b6426a837e239041653ba7becd2e538c076c738025fc" - -[[package]] -name = "pbkdf2" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "216eaa586a190f0a738f2f918511eecfa90f13295abec0e457cdebcceda80cbd" -dependencies = [ - "crypto-mac 0.8.0", -] - -[[package]] -name = "pbkdf2" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d95f5254224e617595d2cc3cc73ff0a5eaf2637519e25f03388154e9378b6ffa" -dependencies = [ - "crypto-mac 0.11.1", -] - -[[package]] -name = "pin-project" -version = "1.0.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58ad3879ad3baf4e44784bc6a718a8698867bb991f8ce24d1bcbe2cfb4c3a75e" -dependencies = [ - "pin-project-internal", -] - -[[package]] -name = "pin-project-internal" -version = "1.0.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744b6f092ba29c3650faf274db506afd39944f48420f6c86b17cfe0ee1cb36bb" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "pin-project-lite" -version = "0.2.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0a7ae3ac2f1173085d398531c705756c94a4c56843785df85a60c1a0afac116" - -[[package]] -name = "pin-utils" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" - -[[package]] -name = "polkadot-core-primitives" -version = "0.9.25" -dependencies = [ - "parity-scale-codec", - "parity-util-mem", - "scale-info", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", -] - -[[package]] -name = "ppv-lite86" -version = "0.2.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb9f9e6e233e5c4a35559a617bf40a4ec447db2e84c20b55a6f83167b7e57872" - -[[package]] -name = "predicates" -version = "2.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5aab5be6e4732b473071984b3164dbbfb7a3674d30ea5ff44410b6bcd960c3c" -dependencies = [ - "difflib", - "itertools", - "predicates-core", -] - -[[package]] -name = "predicates-core" -version = "1.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da1c2388b1513e1b605fcec39a95e0a9e8ef088f71443ef37099fa9ae6673fcb" - -[[package]] -name = "predicates-tree" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d86de6de25020a36c6d3643a86d9a6a9f552107c0559c60ea03551b5e16c032" -dependencies = [ - "predicates-core", - "termtree", -] - -[[package]] -name = "primitive-types" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e28720988bff275df1f51b171e1b2a18c30d194c4d2b61defdacecd625a5d94a" -dependencies = [ - "fixed-hash", - "impl-codec", - "impl-serde", - "scale-info", - "uint", -] - -[[package]] -name = "proc-macro-crate" -version = "1.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e17d47ce914bf4de440332250b0edd23ce48c005f59fab39d3335866b114f11a" -dependencies = [ - "thiserror", - "toml", -] - -[[package]] -name = "proc-macro-error" -version = "1.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da25490ff9892aab3fcf7c36f08cfb902dd3e71ca0f9f9517bea02a73a5ce38c" -dependencies = [ - "proc-macro-error-attr", - "proc-macro2", - "quote", - "syn", - "version_check", -] - -[[package]] -name = "proc-macro-error-attr" -version = "1.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1be40180e52ecc98ad80b184934baf3d0d29f979574e439af5a55274b35f869" -dependencies = [ - "proc-macro2", - "quote", - "version_check", -] - -[[package]] -name = "proc-macro2" -version = "1.0.37" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec757218438d5fda206afc041538b2f6d889286160d649a86a24d37e1235afd1" -dependencies = [ - "unicode-xid", -] - -[[package]] -name = "quote" -version = "1.0.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1feb54ed693b93a84e14094943b84b7c4eae204c512b7ccb95ab0c66d278ad1" -dependencies = [ - "proc-macro2", -] - -[[package]] -name = "radium" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" - -[[package]] -name = "rand" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a6b1679d49b24bbfe0c803429aa1874472f50d9b363131f0e89fc356b544d03" -dependencies = [ - "getrandom 0.1.16", - "libc", - "rand_chacha 0.2.2", - "rand_core 0.5.1", - "rand_hc", - "rand_pcg", -] - -[[package]] -name = "rand" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" -dependencies = [ - "libc", - "rand_chacha 0.3.1", - "rand_core 0.6.3", -] - -[[package]] -name = "rand_chacha" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4c8ed856279c9737206bf725bf36935d8666ead7aa69b52be55af369d193402" -dependencies = [ - "ppv-lite86", - "rand_core 0.5.1", -] - -[[package]] -name = "rand_chacha" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" -dependencies = [ - "ppv-lite86", - "rand_core 0.6.3", -] - -[[package]] -name = "rand_core" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bde5296fc891b0cef12a6d03ddccc162ce7b2aff54160af9338f8d40df6d19" -dependencies = [ - "getrandom 0.1.16", -] - -[[package]] -name = "rand_core" -version = "0.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d34f1408f55294453790c48b2f1ebbb1c5b4b7563eb1f418bcfcfdbb06ebb4e7" -dependencies = [ - "getrandom 0.2.6", -] - -[[package]] -name = "rand_distr" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32cb0b9bc82b0a0876c2dd994a7e7a2683d3e7390ca40e6886785ef0c7e3ee31" -dependencies = [ - "num-traits", - "rand 0.8.5", -] - -[[package]] -name = "rand_hc" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3129af7b92a17112d59ad498c6f81eaf463253766b90396d39ea7a39d6613c" -dependencies = [ - "rand_core 0.5.1", -] - -[[package]] -name = "rand_pcg" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16abd0c1b639e9eb4d7c50c0b8100b0d0f849be2349829c740fe8e6eb4816429" -dependencies = [ - "rand_core 0.5.1", -] - -[[package]] -name = "rawpointer" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60a357793950651c4ed0f3f52338f53b2f809f32d83a07f72909fa13e4c6c1e3" - -[[package]] -name = "redox_syscall" -version = "0.2.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f25bc4c7e55e0b0b7a1d43fb893f4fa1361d0abe38b9ce4f323c2adfe6ef42" -dependencies = [ - "bitflags", -] - -[[package]] -name = "ref-cast" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "685d58625b6c2b83e4cc88a27c4bf65adb7b6b16dbdc413e515c9405b47432ab" -dependencies = [ - "ref-cast-impl", -] - -[[package]] -name = "ref-cast-impl" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a043824e29c94169374ac5183ac0ed43f5724dc4556b19568007486bd840fa1f" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "regex" -version = "1.5.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a11647b6b25ff05a515cb92c365cec08801e83423a235b51e231e1808747286" -dependencies = [ - "aho-corasick", - "memchr", - "regex-syntax", -] - -[[package]] -name = "regex-automata" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" -dependencies = [ - "regex-syntax", -] - -[[package]] -name = "regex-syntax" -version = "0.6.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f497285884f3fcff424ffc933e56d7cbca511def0c9831a7f9b5f6153e3cc89b" - -[[package]] -name = "remote-externalities" -version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "env_logger", - "jsonrpsee", - "log", - "parity-scale-codec", - "serde", - "serde_json", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-version", -] - -[[package]] -name = "rfc6979" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96ef608575f6392792f9ecf7890c00086591d29a83910939d430753f7c050525" -dependencies = [ - "crypto-bigint", - "hmac 0.11.0", - "zeroize", -] - -[[package]] -name = "ring" -version = "0.16.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3053cf52e236a3ed746dfc745aa9cacf1b791d846bdaf412f60a8d7d6e17c8fc" -dependencies = [ - "cc", - "libc", - "once_cell", - "spin", - "untrusted", - "web-sys", - "winapi", -] - -[[package]] -name = "rustc-demangle" -version = "0.1.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ef03e0a2b150c7a90d01faf6254c9c48a41e95fb2a8c2ac1c6f0d2b9aefc342" - -[[package]] -name = "rustc-hash" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" - -[[package]] -name = "rustc-hex" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e75f6a532d0fd9f7f13144f392b6ad56a32696bfcd9c78f797f16bbb6f072d6" - -[[package]] -name = "rustls" -version = "0.20.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fbfeb8d0ddb84706bc597a5574ab8912817c52a397f819e5b614e2265206921" -dependencies = [ - "log", - "ring", - "sct", - "webpki", -] - -[[package]] -name = "rustls-native-certs" -version = "0.6.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0167bac7a9f490495f3c33013e7722b53cb087ecbe082fb0c6387c96f634ea50" -dependencies = [ - "openssl-probe", - "rustls-pemfile", - "schannel", - "security-framework", -] - -[[package]] -name = "rustls-pemfile" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7522c9de787ff061458fe9a829dc790a3f5b22dc571694fc5883f448b94d9a9" -dependencies = [ - "base64", -] - -[[package]] -name = "rustversion" -version = "1.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2cc38e8fa666e2de3c4aba7edeb5ffc5246c1c2ed0e3d17e560aeeba736b23f" - -[[package]] -name = "ryu" -version = "1.0.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73b4b750c782965c211b42f022f59af1fbceabdd026623714f104152f1ec149f" - -[[package]] -name = "sc-transaction-pool-api" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "futures", - "log", - "serde", - "sp-blockchain", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "thiserror", -] - -[[package]] -name = "scale-info" -version = "2.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8980cafbe98a7ee7a9cc16b32ebce542c77883f512d83fbf2ddc8f6a85ea74c9" -dependencies = [ - "bitvec", - "cfg-if", - "derive_more", - "parity-scale-codec", - "scale-info-derive", - "serde", -] - -[[package]] -name = "scale-info-derive" -version = "2.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4260c630e8a8a33429d1688eff2f163f24c65a4e1b1578ef6b565061336e4b6f" -dependencies = [ - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "schannel" -version = "0.1.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f05ba609c234e60bee0d547fe94a4c7e9da733d1c962cf6e59efa4cd9c8bc75" -dependencies = [ - "lazy_static", - "winapi", -] - -[[package]] -name = "schnorrkel" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "021b403afe70d81eea68f6ea12f6b3c9588e5d536a94c3bf80f15e7faa267862" -dependencies = [ - "arrayref", - "arrayvec 0.5.2", - "curve25519-dalek 2.1.3", - "getrandom 0.1.16", - "merlin", - "rand 0.7.3", - "rand_core 0.5.1", - "sha2 0.8.2", - "subtle", - "zeroize", -] - -[[package]] -name = "scopeguard" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" - -[[package]] -name = "sct" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" -dependencies = [ - "ring", - "untrusted", -] - -[[package]] -name = "sec1" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08da66b8b0965a5555b6bd6639e68ccba85e1e2506f5fbb089e93f8a04e1a2d1" -dependencies = [ - "der", - "generic-array 0.14.5", - "subtle", - "zeroize", -] - -[[package]] -name = "secp256k1" -version = "0.21.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c42e6f1735c5f00f51e43e28d6634141f2bcad10931b2609ddd74a86d751260" -dependencies = [ - "secp256k1-sys", -] - -[[package]] -name = "secp256k1-sys" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "957da2573cde917463ece3570eab4a0b3f19de6f1646cde62e6fd3868f566036" -dependencies = [ - "cc", -] - -[[package]] -name = "secrecy" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9bd1c54ea06cfd2f6b63219704de0b9b4f72dcc2b8fdef820be6cd799780e91e" -dependencies = [ - "zeroize", -] - -[[package]] -name = "security-framework" -version = "2.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dc14f172faf8a0194a3aded622712b0de276821addc574fa54fc0a1167e10dc" -dependencies = [ - "bitflags", - "core-foundation", - "core-foundation-sys", - "libc", - "security-framework-sys", -] - -[[package]] -name = "security-framework-sys" -version = "2.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0160a13a177a45bfb43ce71c01580998474f556ad854dcbca936dd2841a5c556" -dependencies = [ - "core-foundation-sys", - "libc", -] - -[[package]] -name = "separator" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f97841a747eef040fcd2e7b3b9a220a7205926e60488e673d9e4926d27772ce5" - -[[package]] -name = "serde" -version = "1.0.137" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61ea8d54c77f8315140a05f4c7237403bf38b72704d031543aa1d16abbf517d1" -dependencies = [ - "serde_derive", -] - -[[package]] -name = "serde_derive" -version = "1.0.137" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f26faba0c3959972377d3b2d306ee9f71faee9714294e41bb777f83f88578be" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "serde_json" -version = "1.0.81" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7ce2b32a1aed03c558dc61a5cd328f15aff2dbc17daad8fb8af04d2100e15c" -dependencies = [ - "itoa 1.0.1", - "ryu", - "serde", -] - -[[package]] -name = "sha-1" -version = "0.9.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99cd6713db3cf16b6c84e06321e049a9b9f699826e16096d23bbcc44d15d51a6" -dependencies = [ - "block-buffer 0.9.0", - "cfg-if", - "cpufeatures", - "digest 0.9.0", - "opaque-debug 0.3.0", -] - -[[package]] -name = "sha2" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a256f46ea78a0c0d9ff00077504903ac881a1dafdc20da66545699e7776b3e69" -dependencies = [ - "block-buffer 0.7.3", - "digest 0.8.1", - "fake-simd", - "opaque-debug 0.2.3", -] - -[[package]] -name = "sha2" -version = "0.9.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d58a1e1bf39749807d89cf2d98ac2dfa0ff1cb3faa38fbb64dd88ac8013d800" -dependencies = [ - "block-buffer 0.9.0", - "cfg-if", - "cpufeatures", - "digest 0.9.0", - "opaque-debug 0.3.0", -] - -[[package]] -name = "sha2" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55deaec60f81eefe3cce0dc50bda92d6d8e88f2a27df7c5033b42afeb1ed2676" -dependencies = [ - "cfg-if", - "cpufeatures", - "digest 0.10.3", -] - -[[package]] -name = "sha3" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "881bf8156c87b6301fc5ca6b27f11eeb2761224c7081e69b409d5a1951a70c86" -dependencies = [ - "digest 0.10.3", - "keccak", -] - -[[package]] -name = "sharded-slab" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "900fba806f70c630b0a382d0d825e17a0f19fcd059a2ade1ff237bcddf446b31" -dependencies = [ - "lazy_static", -] - -[[package]] -name = "signature" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02658e48d89f2bec991f9a78e69cfa4c316f8d6a6c4ec12fae1aeb263d486788" -dependencies = [ - "digest 0.9.0", - "rand_core 0.6.3", -] - -[[package]] -name = "simba" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e82063457853d00243beda9952e910b82593e4b07ae9f721b9278a99a0d3d5c" -dependencies = [ - "approx", - "num-complex", - "num-traits", - "paste", -] - -[[package]] -name = "slab" -version = "0.4.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" - -[[package]] -name = "smallvec" -version = "1.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2dd574626839106c320a323308629dcb1acfc96e32a8cba364ddc61ac23ee83" - -[[package]] -name = "socket2" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66d72b759436ae32898a2af0a14218dbf55efde3feeb170eb623637db85ee1e0" -dependencies = [ - "libc", - "winapi", -] - -[[package]] -name = "soketto" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41d1c5305e39e09653383c2c7244f2f78b3bcae37cf50c64cb4789c9f5096ec2" -dependencies = [ - "base64", - "bytes", - "futures", - "httparse", - "log", - "rand 0.8.5", - "sha-1", -] - -[[package]] -name = "sp-api" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "hash-db", - "log", - "parity-scale-codec", - "sp-api-proc-macro", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-state-machine 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-version", - "thiserror", -] - -[[package]] -name = "sp-api-proc-macro" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "blake2", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-application-crypto" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acb4490364cb3b097a6755343e552495b0013778152300714be4647d107e9a2e" -dependencies = [ - "parity-scale-codec", - "scale-info", - "serde", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-io 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "sp-application-crypto" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "parity-scale-codec", - "scale-info", - "serde", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "sp-application-crypto" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "parity-scale-codec", - "scale-info", - "serde", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", -] - -[[package]] -name = "sp-arithmetic" -version = "5.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31ef21f82cc10f75ed046b65e2f8048080ee76e59f1b8aed55c7150daebfd35b" -dependencies = [ - "integer-sqrt", - "num-traits", - "parity-scale-codec", - "scale-info", - "serde", - "sp-debug-derive 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "static_assertions", -] - -[[package]] -name = "sp-arithmetic" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "integer-sqrt", - "num-traits", - "parity-scale-codec", - "scale-info", - "serde", - "sp-debug-derive 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "static_assertions", -] - -[[package]] -name = "sp-arithmetic" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "integer-sqrt", - "num-traits", - "parity-scale-codec", - "scale-info", - "serde", - "sp-debug-derive 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "static_assertions", -] - -[[package]] -name = "sp-blockchain" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "futures", - "log", - "lru", - "parity-scale-codec", - "parking_lot", - "sp-api", - "sp-consensus", - "sp-database", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-state-machine 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "thiserror", -] - -[[package]] -name = "sp-consensus" -version = "0.10.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "async-trait", - "futures", - "futures-timer", - "log", - "parity-scale-codec", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-inherents", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-state-machine 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-version", - "thiserror", -] - -[[package]] -name = "sp-core" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77963e2aa8fadb589118c3aede2e78b6c4bcf1c01d588fbf33e915b390825fbd" -dependencies = [ - "base58", - "bitflags", - "blake2-rfc", - "byteorder", - "dyn-clonable", - "ed25519-dalek", - "futures", - "hash-db", - "hash256-std-hasher", - "hex", - "impl-serde", - "lazy_static", - "libsecp256k1", - "log", - "merlin", - "num-traits", - "parity-scale-codec", - "parity-util-mem", - "parking_lot", - "primitive-types", - "rand 0.7.3", - "regex", - "scale-info", - "schnorrkel", - "secp256k1", - "secrecy", - "serde", - "sp-core-hashing 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-debug-derive 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-externalities 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-runtime-interface 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-storage 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "ss58-registry", - "substrate-bip39", - "thiserror", - "tiny-bip39", - "wasmi", - "zeroize", -] - -[[package]] -name = "sp-core" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "base58", - "bitflags", - "blake2-rfc", - "byteorder", - "dyn-clonable", - "ed25519-dalek", - "futures", - "hash-db", - "hash256-std-hasher", - "hex", - "impl-serde", - "lazy_static", - "libsecp256k1", - "log", - "merlin", - "num-traits", - "parity-scale-codec", - "parity-util-mem", - "parking_lot", - "primitive-types", - "rand 0.7.3", - "regex", - "scale-info", - "schnorrkel", - "secp256k1", - "secrecy", - "serde", - "sp-core-hashing 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-debug-derive 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "ss58-registry", - "substrate-bip39", - "thiserror", - "tiny-bip39", - "wasmi", - "zeroize", -] - -[[package]] -name = "sp-core" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "base58", - "bitflags", - "blake2-rfc", - "byteorder", - "dyn-clonable", - "ed25519-dalek", - "futures", - "hash-db", - "hash256-std-hasher", - "hex", - "impl-serde", - "lazy_static", - "libsecp256k1", - "log", - "merlin", - "num-traits", - "parity-scale-codec", - "parity-util-mem", - "parking_lot", - "primitive-types", - "rand 0.7.3", - "regex", - "scale-info", - "schnorrkel", - "secp256k1", - "secrecy", - "serde", - "sp-core-hashing 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-debug-derive 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-runtime-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "ss58-registry", - "substrate-bip39", - "thiserror", - "tiny-bip39", - "wasmi", - "zeroize", -] - -[[package]] -name = "sp-core-hashing" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec864a6a67249f0c8dd3d5acab43623a61677e85ff4f2f9b04b802d2fe780e83" -dependencies = [ - "blake2-rfc", - "byteorder", - "sha2 0.9.9", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "tiny-keccak", - "twox-hash", -] - -[[package]] -name = "sp-core-hashing" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "blake2", - "byteorder", - "digest 0.10.3", - "sha2 0.10.2", - "sha3", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "twox-hash", -] - -[[package]] -name = "sp-core-hashing" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "blake2", - "byteorder", - "digest 0.10.3", - "sha2 0.10.2", - "sha3", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "twox-hash", -] - -[[package]] -name = "sp-core-hashing-proc-macro" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "proc-macro2", - "quote", - "sp-core-hashing 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "syn", -] - -[[package]] -name = "sp-database" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "kvdb", - "parking_lot", -] - -[[package]] -name = "sp-debug-derive" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d676664972e22a0796176e81e7bec41df461d1edf52090955cdab55f2c956ff2" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-debug-derive" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-debug-derive" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-externalities" -version = "0.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fcfd91f92a2a59224230a77c4a5d6f51709620c0aab4e51f108ccece6adc56f" -dependencies = [ - "environmental", - "parity-scale-codec", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-storage 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "sp-externalities" -version = "0.12.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "environmental", - "parity-scale-codec", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "sp-externalities" -version = "0.12.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "environmental", - "parity-scale-codec", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", -] - -[[package]] -name = "sp-inherents" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "async-trait", - "impl-trait-for-tuples", - "parity-scale-codec", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "thiserror", -] - -[[package]] -name = "sp-io" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "935fd3c71bad6811a7984cabb74d323b8ca3107024024c3eabb610e0182ba8d3" -dependencies = [ - "futures", - "hash-db", - "libsecp256k1", - "log", - "parity-scale-codec", - "parking_lot", - "secp256k1", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-externalities 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-keystore 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-runtime-interface 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-state-machine 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-tracing 5.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-trie 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-wasm-interface 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "tracing", - "tracing-core", -] - -[[package]] -name = "sp-io" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "futures", - "hash-db", - "libsecp256k1", - "log", - "parity-scale-codec", - "parking_lot", - "secp256k1", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-keystore 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-state-machine 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-tracing 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-trie 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-wasm-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "tracing", - "tracing-core", -] - -[[package]] -name = "sp-io" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "futures", - "hash-db", - "libsecp256k1", - "log", - "parity-scale-codec", - "parking_lot", - "secp256k1", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-keystore 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-runtime-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-state-machine 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-tracing 5.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-trie 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-wasm-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "tracing", - "tracing-core", -] - -[[package]] -name = "sp-keystore" -version = "0.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3261eddca8c8926e3e1de136a7980cb3afc3455247d9d6f3119d9b292f73aaee" -dependencies = [ - "async-trait", - "futures", - "merlin", - "parity-scale-codec", - "parking_lot", - "schnorrkel", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-externalities 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "thiserror", -] - -[[package]] -name = "sp-keystore" -version = "0.12.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "async-trait", - "futures", - "merlin", - "parity-scale-codec", - "parking_lot", - "schnorrkel", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "thiserror", -] - -[[package]] -name = "sp-keystore" -version = "0.12.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "async-trait", - "futures", - "merlin", - "parity-scale-codec", - "parking_lot", - "schnorrkel", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "thiserror", -] - -[[package]] -name = "sp-npos-elections" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "parity-scale-codec", - "scale-info", - "serde", - "sp-arithmetic 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "sp-panic-handler" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2101f3c555fceafcfcfb0e61c55ea9ed80dc60bd77d54d9f25b369edb029e9a4" -dependencies = [ - "backtrace", - "lazy_static", - "regex", -] - -[[package]] -name = "sp-panic-handler" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "backtrace", - "lazy_static", - "regex", -] - -[[package]] -name = "sp-panic-handler" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "backtrace", - "lazy_static", - "regex", -] - -[[package]] -name = "sp-runtime" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb7d8a8d5ab5d349c6cf9300af1721b7b6446ba63401dbb11c10a1d65197aa5f" -dependencies = [ - "either", - "hash256-std-hasher", - "impl-trait-for-tuples", - "log", - "parity-scale-codec", - "parity-util-mem", - "paste", - "rand 0.7.3", - "scale-info", - "serde", - "sp-application-crypto 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-arithmetic 5.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-io 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "sp-runtime" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "either", - "hash256-std-hasher", - "impl-trait-for-tuples", - "log", - "parity-scale-codec", - "parity-util-mem", - "paste", - "rand 0.7.3", - "scale-info", - "serde", - "sp-application-crypto 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-arithmetic 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "sp-runtime" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "either", - "hash256-std-hasher", - "impl-trait-for-tuples", - "log", - "parity-scale-codec", - "parity-util-mem", - "paste", - "rand 0.7.3", - "scale-info", - "serde", - "sp-application-crypto 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-arithmetic 5.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", -] - -[[package]] -name = "sp-runtime-interface" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "158bf0305c75a50fc0e334b889568f519a126e32b87900c3f4251202dece7b4b" -dependencies = [ - "impl-trait-for-tuples", - "parity-scale-codec", - "primitive-types", - "sp-externalities 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-runtime-interface-proc-macro 5.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-storage 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-tracing 5.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-wasm-interface 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "static_assertions", -] - -[[package]] -name = "sp-runtime-interface" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "impl-trait-for-tuples", - "parity-scale-codec", - "primitive-types", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-runtime-interface-proc-macro 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-tracing 5.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-wasm-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "static_assertions", -] - -[[package]] -name = "sp-runtime-interface" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "impl-trait-for-tuples", - "parity-scale-codec", - "primitive-types", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-runtime-interface-proc-macro 5.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-storage 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-tracing 5.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-wasm-interface 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "static_assertions", -] - -[[package]] -name = "sp-runtime-interface-proc-macro" -version = "5.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ecb916b9664ed9f90abef0ff5a3e61454c1efea5861b2997e03f39b59b955f" -dependencies = [ - "Inflector", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-runtime-interface-proc-macro" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "Inflector", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-runtime-interface-proc-macro" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "Inflector", - "proc-macro-crate", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-staking" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "parity-scale-codec", - "scale-info", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "sp-state-machine" -version = "0.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecee3b33eb78c99997676a571656bcc35db6886abecfddd13e76a73b5871c6c1" -dependencies = [ - "hash-db", - "log", - "num-traits", - "parity-scale-codec", - "parking_lot", - "rand 0.7.3", - "smallvec", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-externalities 0.12.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-panic-handler 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-trie 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "thiserror", - "tracing", - "trie-db", - "trie-root", -] - -[[package]] -name = "sp-state-machine" -version = "0.12.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "hash-db", - "log", - "num-traits", - "parity-scale-codec", - "parking_lot", - "rand 0.7.3", - "smallvec", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-panic-handler 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-trie 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "thiserror", - "tracing", - "trie-root", -] - -[[package]] -name = "sp-state-machine" -version = "0.12.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "hash-db", - "log", - "num-traits", - "parity-scale-codec", - "parking_lot", - "rand 0.7.3", - "smallvec", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-externalities 0.12.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-panic-handler 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-trie 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "thiserror", - "tracing", - "trie-root", -] - -[[package]] -name = "sp-std" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14804d6069ee7a388240b665f17908d98386ffb0b5d39f89a4099fc7a2a4c03f" - -[[package]] -name = "sp-std" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" - -[[package]] -name = "sp-std" -version = "4.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" - -[[package]] -name = "sp-storage" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dab53af846068e3e0716d3ccc70ea0db44035c79b2ed5821aaa6635039efa37" -dependencies = [ - "impl-serde", - "parity-scale-codec", - "ref-cast", - "serde", - "sp-debug-derive 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "sp-storage" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "impl-serde", - "parity-scale-codec", - "ref-cast", - "serde", - "sp-debug-derive 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", -] - -[[package]] -name = "sp-storage" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "impl-serde", - "parity-scale-codec", - "ref-cast", - "serde", - "sp-debug-derive 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", -] - -[[package]] -name = "sp-tracing" -version = "5.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69a67e555d171c4238bd223393cda747dd20ec7d4f5fe5c042c056cb7fde9eda" -dependencies = [ - "parity-scale-codec", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "tracing", - "tracing-core", - "tracing-subscriber 0.2.25", -] - -[[package]] -name = "sp-tracing" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "parity-scale-codec", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "tracing", - "tracing-core", - "tracing-subscriber 0.2.25", -] - -[[package]] -name = "sp-tracing" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "parity-scale-codec", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "tracing", - "tracing-core", - "tracing-subscriber 0.2.25", -] - -[[package]] -name = "sp-trie" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6fc34f4f291886914733e083b62708d829f3e6b8d7a7ca7fa8a55a3d7640b0b" -dependencies = [ - "hash-db", - "memory-db", - "parity-scale-codec", - "scale-info", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "trie-db", - "trie-root", -] - -[[package]] -name = "sp-trie" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "hash-db", - "memory-db", - "parity-scale-codec", - "scale-info", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "thiserror", - "trie-db", - "trie-root", -] - -[[package]] -name = "sp-trie" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "hash-db", - "memory-db", - "parity-scale-codec", - "scale-info", - "sp-core 6.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "thiserror", - "trie-db", - "trie-root", -] - -[[package]] -name = "sp-version" -version = "5.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "impl-serde", - "parity-scale-codec", - "parity-wasm", - "scale-info", - "serde", - "sp-core-hashing-proc-macro", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-version-proc-macro", - "thiserror", -] - -[[package]] -name = "sp-version-proc-macro" -version = "4.0.0-dev" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "parity-scale-codec", - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "sp-wasm-interface" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10d88debe690c2b24eaa9536a150334fcef2ae184c21a0e5b3e80135407a7d52" -dependencies = [ - "impl-trait-for-tuples", - "log", - "parity-scale-codec", - "sp-std 4.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "wasmi", -] - -[[package]] -name = "sp-wasm-interface" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait#7227aa3df1b765dc850eb1dd4feb05e01805d9e0" -dependencies = [ - "impl-trait-for-tuples", - "log", - "parity-scale-codec", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "wasmi", -] - -[[package]] -name = "sp-wasm-interface" -version = "6.0.0" -source = "git+https://github.com/paritytech/substrate?branch=master#c2fc4b3ca0d7a15cc3f9cb1e5f441d99ec8d6e0b" -dependencies = [ - "impl-trait-for-tuples", - "log", - "parity-scale-codec", - "sp-std 4.0.0 (git+https://github.com/paritytech/substrate?branch=master)", - "wasmi", -] - -[[package]] -name = "spin" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" - -[[package]] -name = "ss58-registry" -version = "1.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b84a70894df7a73666e0694f44b41a9571625e9546fb58a0818a565d2c7e084" -dependencies = [ - "Inflector", - "num-format", - "proc-macro2", - "quote", - "serde", - "serde_json", - "unicode-xid", -] - -[[package]] -name = "staking-miner" -version = "0.9.25" -dependencies = [ - "anyhow", - "assert_cmd", - "clap", - "frame-election-provider-support", - "frame-support", - "frame-system", - "jsonrpsee", - "log", - "pallet-election-provider-multi-phase", - "pallet-transaction-payment", - "parity-scale-codec", - "paste", - "polkadot-core-primitives", - "remote-externalities", - "sc-transaction-pool-api", - "scale-info", - "serde", - "serde_json", - "sp-io 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-npos-elections", - "sp-runtime 6.0.0 (git+https://github.com/paritytech/substrate?branch=kiz-staking-miner-new-trait)", - "sp-version", - "sub-tokens", - "subxt", - "thiserror", - "tokio", - "tracing-subscriber 0.3.11", -] - -[[package]] -name = "static_assertions" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" - -[[package]] -name = "statrs" -version = "0.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05bdbb8e4e78216a85785a85d3ec3183144f98d0097b9281802c019bb07a6f05" -dependencies = [ - "approx", - "lazy_static", - "nalgebra", - "num-traits", - "rand 0.8.5", -] - -[[package]] -name = "strsim" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" - -[[package]] -name = "strum" -version = "0.23.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cae14b91c7d11c9a851d3fbc80a963198998c2a64eec840477fa92d8ce9b70bb" -dependencies = [ - "strum_macros", -] - -[[package]] -name = "strum_macros" -version = "0.23.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bb0dc7ee9c15cea6199cde9a127fa16a4c5819af85395457ad72d68edc85a38" -dependencies = [ - "heck 0.3.3", - "proc-macro2", - "quote", - "rustversion", - "syn", -] - -[[package]] -name = "sub-tokens" -version = "0.1.0" -source = "git+https://github.com/paritytech/substrate-debug-kit?branch=master#e12503ab781e913735dc389865a3b8b4a6c6399d" -dependencies = [ - "separator", -] - -[[package]] -name = "substrate-bip39" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49eee6965196b32f882dd2ee85a92b1dbead41b04e53907f269de3b0dc04733c" -dependencies = [ - "hmac 0.11.0", - "pbkdf2 0.8.0", - "schnorrkel", - "sha2 0.9.9", - "zeroize", -] - -[[package]] -name = "subtle" -version = "2.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" - -[[package]] -name = "subxt" -version = "0.21.0" -source = "git+https://github.com/paritytech/subxt#6d73172560b4c722aeac19695c96c618799d50a2" -dependencies = [ - "bitvec", - "derivative", - "frame-metadata", - "futures", - "hex", - "jsonrpsee", - "log", - "parity-scale-codec", - "parking_lot", - "scale-info", - "serde", - "serde_json", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "sp-runtime 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "subxt-macro", - "subxt-metadata", - "thiserror", -] - -[[package]] -name = "subxt-codegen" -version = "0.21.0" -source = "git+https://github.com/paritytech/subxt#6d73172560b4c722aeac19695c96c618799d50a2" -dependencies = [ - "darling", - "frame-metadata", - "heck 0.4.0", - "parity-scale-codec", - "proc-macro-error", - "proc-macro2", - "quote", - "scale-info", - "subxt-metadata", - "syn", -] - -[[package]] -name = "subxt-macro" -version = "0.21.0" -source = "git+https://github.com/paritytech/subxt#6d73172560b4c722aeac19695c96c618799d50a2" -dependencies = [ - "darling", - "proc-macro-error", - "subxt-codegen", - "syn", -] - -[[package]] -name = "subxt-metadata" -version = "0.21.0" -source = "git+https://github.com/paritytech/subxt#6d73172560b4c722aeac19695c96c618799d50a2" -dependencies = [ - "frame-metadata", - "parity-scale-codec", - "scale-info", - "sp-core 6.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "syn" -version = "1.0.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ff7c592601f11445996a06f8ad0c27f094a58857c2f89e97974ab9235b92c52" -dependencies = [ - "proc-macro2", - "quote", - "unicode-xid", -] - -[[package]] -name = "synstructure" -version = "0.12.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f36bdaa60a83aca3921b5259d5400cbf5e90fc51931376a9bd4a0eb79aa7210f" -dependencies = [ - "proc-macro2", - "quote", - "syn", - "unicode-xid", -] - -[[package]] -name = "tap" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" - -[[package]] -name = "termcolor" -version = "1.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bab24d30b911b2376f3a13cc2cd443142f0c81dda04c118693e35b3835757755" -dependencies = [ - "winapi-util", -] - -[[package]] -name = "termtree" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "507e9898683b6c43a9aa55b64259b721b52ba226e0f3779137e50ad114a4c90b" - -[[package]] -name = "textwrap" -version = "0.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1141d4d61095b28419e22cb0bbf02755f5e54e0526f97f1e3d1d160e60885fb" - -[[package]] -name = "thiserror" -version = "1.0.31" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd829fe32373d27f76265620b5309d0340cb8550f523c1dda251d6298069069a" -dependencies = [ - "thiserror-impl", -] - -[[package]] -name = "thiserror-impl" -version = "1.0.31" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0396bc89e626244658bef819e22d0cc459e795a5ebe878e6ec336d1674a8d79a" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "thread_local" -version = "1.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5516c27b78311c50bf42c071425c560ac799b11c30b31f87e3081965fe5e0180" -dependencies = [ - "once_cell", -] - -[[package]] -name = "tiny-bip39" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffc59cb9dfc85bb312c3a78fd6aa8a8582e310b0fa885d5bb877f6dcc601839d" -dependencies = [ - "anyhow", - "hmac 0.8.1", - "once_cell", - "pbkdf2 0.4.0", - "rand 0.7.3", - "rustc-hash", - "sha2 0.9.9", - "thiserror", - "unicode-normalization", - "wasm-bindgen", - "zeroize", -] - -[[package]] -name = "tiny-keccak" -version = "2.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" -dependencies = [ - "crunchy", -] - -[[package]] -name = "tinyvec" -version = "1.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" -dependencies = [ - "tinyvec_macros", -] - -[[package]] -name = "tinyvec_macros" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" - -[[package]] -name = "tokio" -version = "1.18.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dce653fb475565de9f6fb0614b28bca8df2c430c0cf84bcd9c843f15de5414cc" -dependencies = [ - "libc", - "mio", - "num_cpus", - "once_cell", - "pin-project-lite", - "socket2", - "tokio-macros", - "winapi", -] - -[[package]] -name = "tokio-macros" -version = "1.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b557f72f448c511a979e2564e55d74e6c4432fc96ff4f6241bc6bded342643b7" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "tokio-rustls" -version = "0.23.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4151fda0cf2798550ad0b34bcfc9b9dcc2a9d2471c895c68f3a8818e54f2389e" -dependencies = [ - "rustls", - "tokio", - "webpki", -] - -[[package]] -name = "tokio-util" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0edfdeb067411dba2044da6d1cb2df793dd35add7888d73c16e3381ded401764" -dependencies = [ - "bytes", - "futures-core", - "futures-io", - "futures-sink", - "pin-project-lite", - "tokio", -] - -[[package]] -name = "toml" -version = "0.5.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d82e1a7758622a465f8cee077614c73484dac5b836c02ff6a40d5d1010324d7" -dependencies = [ - "serde", -] - -[[package]] -name = "tower-service" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" - -[[package]] -name = "tracing" -version = "0.1.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d0ecdcb44a79f0fe9844f0c4f33a342cbcbb5117de8001e6ba0dc2351327d09" -dependencies = [ - "cfg-if", - "pin-project-lite", - "tracing-attributes", - "tracing-core", -] - -[[package]] -name = "tracing-attributes" -version = "0.1.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6b8ad3567499f98a1db7a752b07a7c8c7c7c34c332ec00effb2b0027974b7c" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "tracing-core" -version = "0.1.26" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f54c8ca710e81886d498c2fd3331b56c93aa248d49de2222ad2742247c60072f" -dependencies = [ - "lazy_static", - "valuable", -] - -[[package]] -name = "tracing-log" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78ddad33d2d10b1ed7eb9d1f518a5674713876e97e5bb9b7345a7984fbb4f922" -dependencies = [ - "lazy_static", - "log", - "tracing-core", -] - -[[package]] -name = "tracing-serde" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc6b213177105856957181934e4920de57730fc69bf42c37ee5bb664d406d9e1" -dependencies = [ - "serde", - "tracing-core", -] - -[[package]] -name = "tracing-subscriber" -version = "0.2.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e0d2eaa99c3c2e41547cfa109e910a68ea03823cccad4a0525dcbc9b01e8c71" -dependencies = [ - "ansi_term", - "chrono", - "lazy_static", - "matchers 0.0.1", - "regex", - "serde", - "serde_json", - "sharded-slab", - "smallvec", - "thread_local", - "tracing", - "tracing-core", - "tracing-log", - "tracing-serde", -] - -[[package]] -name = "tracing-subscriber" -version = "0.3.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bc28f93baff38037f64e6f43d34cfa1605f27a49c34e8a04c5e78b0babf2596" -dependencies = [ - "ansi_term", - "lazy_static", - "matchers 0.1.0", - "regex", - "sharded-slab", - "smallvec", - "thread_local", - "tracing", - "tracing-core", - "tracing-log", -] - -[[package]] -name = "trie-db" -version = "0.23.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d32d034c0d3db64b43c31de38e945f15b40cd4ca6d2dcfc26d4798ce8de4ab83" -dependencies = [ - "hash-db", - "hashbrown 0.12.1", - "log", - "rustc-hex", - "smallvec", -] - -[[package]] -name = "trie-root" -version = "0.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a36c5ca3911ed3c9a5416ee6c679042064b93fc637ded67e25f92e68d783891" -dependencies = [ - "hash-db", -] - -[[package]] -name = "try-lock" -version = "0.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" - -[[package]] -name = "tt-call" -version = "1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e66dcbec4290c69dd03c57e76c2469ea5c7ce109c6dd4351c13055cf71ea055" - -[[package]] -name = "twox-hash" -version = "1.6.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ee73e6e4924fe940354b8d4d98cad5231175d615cd855b758adc658c0aac6a0" -dependencies = [ - "cfg-if", - "digest 0.10.3", - "rand 0.8.5", - "static_assertions", -] - -[[package]] -name = "typenum" -version = "1.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" - -[[package]] -name = "uint" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12f03af7ccf01dd611cc450a0d10dbc9b745770d096473e2faf0ca6e2d66d1e0" -dependencies = [ - "byteorder", - "crunchy", - "hex", - "static_assertions", -] - -[[package]] -name = "unicode-normalization" -version = "0.1.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d54590932941a9e9266f0832deed84ebe1bf2e4c9e4a3554d393d18f5e854bf9" -dependencies = [ - "tinyvec", -] - -[[package]] -name = "unicode-segmentation" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8820f5d777f6224dc4be3632222971ac30164d4a258d595640799554ebfd99" - -[[package]] -name = "unicode-xid" -version = "0.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "957e51f3646910546462e67d5f7599b9e4fb8acdd304b087a6494730f9eebf04" - -[[package]] -name = "untrusted" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" - -[[package]] -name = "valuable" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" - -[[package]] -name = "wait-timeout" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f200f5b12eb75f8c1ed65abd4b2db8a6e1b138a20de009dacee265a2498f3f6" -dependencies = [ - "libc", -] - -[[package]] -name = "want" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ce8a968cb1cd110d136ff8b819a556d6fb6d919363c61534f6860c7eb172ba0" -dependencies = [ - "log", - "try-lock", -] - -[[package]] -name = "wasi" -version = "0.9.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cccddf32554fecc6acb585f82a32a72e28b48f8c4c1883ddfeeeaa96f7d8e519" - -[[package]] -name = "wasi" -version = "0.10.2+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd6fbd9a79829dd1ad0cc20627bf1ed606756a7f77edff7b66b7064f9cb327c6" - -[[package]] -name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" - -[[package]] -name = "wasm-bindgen" -version = "0.2.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27370197c907c55e3f1a9fbe26f44e937fe6451368324e009cba39e139dc08ad" -dependencies = [ - "cfg-if", - "wasm-bindgen-macro", -] - -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53e04185bfa3a779273da532f5025e33398409573f348985af9a1cbf3774d3f4" -dependencies = [ - "bumpalo", - "lazy_static", - "log", - "proc-macro2", - "quote", - "syn", - "wasm-bindgen-shared", -] - -[[package]] -name = "wasm-bindgen-macro" -version = "0.2.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17cae7ff784d7e83a2fe7611cfe766ecf034111b49deb850a3dc7699c08251f5" -dependencies = [ - "quote", - "wasm-bindgen-macro-support", -] - -[[package]] -name = "wasm-bindgen-macro-support" -version = "0.2.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99ec0dc7a4756fffc231aab1b9f2f578d23cd391390ab27f952ae0c9b3ece20b" -dependencies = [ - "proc-macro2", - "quote", - "syn", - "wasm-bindgen-backend", - "wasm-bindgen-shared", -] - -[[package]] -name = "wasm-bindgen-shared" -version = "0.2.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d554b7f530dee5964d9a9468d95c1f8b8acae4f282807e7d27d4b03099a46744" - -[[package]] -name = "wasmi" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca00c5147c319a8ec91ec1a0edbec31e566ce2c9cc93b3f9bb86a9efd0eb795d" -dependencies = [ - "downcast-rs", - "libc", - "memory_units", - "num-rational 0.2.4", - "num-traits", - "parity-wasm", - "wasmi-validation", -] - -[[package]] -name = "wasmi-validation" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "165343ecd6c018fc09ebcae280752702c9a2ef3e6f8d02f1cfcbdb53ef6d7937" -dependencies = [ - "parity-wasm", -] - -[[package]] -name = "web-sys" -version = "0.3.57" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b17e741662c70c8bd24ac5c5b18de314a2c26c32bf8346ee1e6f53de919c283" -dependencies = [ - "js-sys", - "wasm-bindgen", -] - -[[package]] -name = "webpki" -version = "0.22.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f095d78192e208183081cc07bc5515ef55216397af48b873e5edcd72637fa1bd" -dependencies = [ - "ring", - "untrusted", -] - -[[package]] -name = "webpki-roots" -version = "0.22.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d8de8415c823c8abd270ad483c6feeac771fad964890779f9a8cb24fbbc1bf" -dependencies = [ - "webpki", -] - -[[package]] -name = "winapi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" -dependencies = [ - "winapi-i686-pc-windows-gnu", - "winapi-x86_64-pc-windows-gnu", -] - -[[package]] -name = "winapi-i686-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" - -[[package]] -name = "winapi-util" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70ec6ce85bb158151cae5e5c87f95a8e97d2c0c4b001223f33a334e3ce5de178" -dependencies = [ - "winapi", -] - -[[package]] -name = "winapi-x86_64-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" - -[[package]] -name = "windows-sys" -version = "0.36.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea04155a16a59f9eab786fe12a4a450e75cdb175f9e0d80da1e17db09f55b8d2" -dependencies = [ - "windows_aarch64_msvc", - "windows_i686_gnu", - "windows_i686_msvc", - "windows_x86_64_gnu", - "windows_x86_64_msvc", -] - -[[package]] -name = "windows_aarch64_msvc" -version = "0.36.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9bb8c3fd39ade2d67e9874ac4f3db21f0d710bee00fe7cab16949ec184eeaa47" - -[[package]] -name = "windows_i686_gnu" -version = "0.36.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "180e6ccf01daf4c426b846dfc66db1fc518f074baa793aa7d9b9aaeffad6a3b6" - -[[package]] -name = "windows_i686_msvc" -version = "0.36.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2e7917148b2812d1eeafaeb22a97e4813dfa60a3f8f78ebe204bcc88f12f024" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.36.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dcd171b8776c41b97521e5da127a2d86ad280114807d0b2ab1e462bc764d9e1" - -[[package]] -name = "windows_x86_64_msvc" -version = "0.36.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c811ca4a8c853ef420abd8592ba53ddbbac90410fab6903b3e79972a631f7680" - -[[package]] -name = "wyz" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30b31594f29d27036c383b53b59ed3476874d518f0efb151b27a4c275141390e" -dependencies = [ - "tap", -] - -[[package]] -name = "zeroize" -version = "1.5.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94693807d016b2f2d2e14420eb3bfcca689311ff775dcf113d74ea624b7cdf07" -dependencies = [ - "zeroize_derive", -] - -[[package]] -name = "zeroize_derive" -version = "1.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f8f187641dad4f680d25c4bfc4225b418165984179f26ca76ec4fb6441d3a17" -dependencies = [ - "proc-macro2", - "quote", - "syn", - "synstructure", -] diff --git a/xcm/pallet-xcm/src/lib.rs b/xcm/pallet-xcm/src/lib.rs index 914c177399cc..47a9d7d75d0a 100644 --- a/xcm/pallet-xcm/src/lib.rs +++ b/xcm/pallet-xcm/src/lib.rs @@ -157,7 +157,6 @@ pub mod pallet { } #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::storage_version(migration::STORAGE_VERSION)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/xcm/pallet-xcm/src/migration.rs b/xcm/pallet-xcm/src/migration.rs index c7e8ca786244..247b9f8adaca 100644 --- a/xcm/pallet-xcm/src/migration.rs +++ b/xcm/pallet-xcm/src/migration.rs @@ -14,7 +14,7 @@ // You should have received a copy of the GNU General Public License // along with Polkadot. If not, see . -use crate::{Config, Pallet, Store}; +use crate::{Config, Pallet, VersionNotifyTargets}; use frame_support::{ pallet_prelude::*, traits::{OnRuntimeUpgrade, StorageVersion}, @@ -48,7 +48,7 @@ pub mod v1 { Some(translated) }; - as Store>::VersionNotifyTargets::translate_values(translate); + VersionNotifyTargets::::translate_values(translate); log::info!("v1 applied successfully"); STORAGE_VERSION.put::>(); diff --git a/xcm/pallet-xcm/src/mock.rs b/xcm/pallet-xcm/src/mock.rs index 3c9184898f63..74e69c55b31c 100644 --- a/xcm/pallet-xcm/src/mock.rs +++ b/xcm/pallet-xcm/src/mock.rs @@ -51,7 +51,6 @@ pub mod pallet_test_notifier { use xcm::latest::prelude::*; #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] pub struct Pallet(_); #[pallet::config] diff --git a/xcm/xcm-simulator/example/src/parachain.rs b/xcm/xcm-simulator/example/src/parachain.rs index d42f862c21ce..9801a847b712 100644 --- a/xcm/xcm-simulator/example/src/parachain.rs +++ b/xcm/xcm-simulator/example/src/parachain.rs @@ -260,7 +260,6 @@ pub mod mock_msg_queue { impl Pallet {} #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); diff --git a/xcm/xcm-simulator/fuzzer/src/parachain.rs b/xcm/xcm-simulator/fuzzer/src/parachain.rs index f1e3f4a6220e..adf134124c93 100644 --- a/xcm/xcm-simulator/fuzzer/src/parachain.rs +++ b/xcm/xcm-simulator/fuzzer/src/parachain.rs @@ -174,7 +174,6 @@ pub mod mock_msg_queue { impl Pallet {} #[pallet::pallet] - #[pallet::generate_store(pub(super) trait Store)] #[pallet::without_storage_info] pub struct Pallet(_); From db26c64380f09d4d11a9728010d4e0edcd4aa063 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 15 Mar 2023 20:41:21 +0100 Subject: [PATCH 09/16] Add docs; Fix tests --- node/core/pvf/src/execute/worker.rs | 4 +++- node/core/pvf/src/prepare/worker.rs | 4 +++- node/core/pvf/src/testing.rs | 19 ++++++++++++++----- 3 files changed, 20 insertions(+), 7 deletions(-) diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 47a2e65af95a..7ac2c5b0dd22 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -264,7 +264,9 @@ impl Response { } /// The entrypoint that the spawned execute worker should start with. The `socket_path` specifies -/// the path to the socket used to communicate with the host. +/// the path to the socket used to communicate with the host. The `node_version`, if `Some`, +/// is checked against the worker version. A mismatch results in immediate worker termination. +/// `None` is used for tests and in other situations when version check is not necessary. pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { worker_event_loop("execute", socket_path, |rt_handle, mut stream| async move { let worker_pid = std::process::id(); diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index dd88650f7fb9..8d18dd3a072e 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -327,7 +327,9 @@ async fn recv_response(stream: &mut UnixStream, pid: u32) -> io::Result>(); - if args.len() < 2 { + if args.len() < 3 { panic!("wrong number of arguments"); } + let mut version = None; + let mut socket_path: &str = ""; + + for i in 2..args.len() { + match args[i].as_ref() { + "--socket-path" => socket_path = args[i + 1].as_str(), + "--node-version" => version = Some(args[i + 1].as_str()), + _ => (), + } + } + let subcommand = &args[1]; match subcommand.as_ref() { "sleep" => { std::thread::sleep(std::time::Duration::from_secs(5)); }, "prepare-worker" => { - let socket_path = &args[2]; - $crate::prepare_worker_entrypoint(socket_path, None); + $crate::prepare_worker_entrypoint(&socket_path, version); }, "execute-worker" => { - let socket_path = &args[2]; - $crate::execute_worker_entrypoint(socket_path, None); + $crate::execute_worker_entrypoint(&socket_path, version); }, other => panic!("unknown subcommand: {}", other), } From b696baafc66bf0751480a014e6a89cd2a1a55799 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 15 Mar 2023 20:48:01 +0100 Subject: [PATCH 10/16] Update Cargo.lock --- Cargo.lock | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.lock b/Cargo.lock index 35e5b71541df..de1af253cd59 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7057,6 +7057,7 @@ dependencies = [ "sp-maybe-compressed-blob", "sp-tracing", "sp-wasm-interface", + "substrate-build-script-utils", "tempfile", "test-parachain-adder", "test-parachain-halt", From 501d4be666fb58a5410e4cc153a51344e61529d4 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Sat, 18 Mar 2023 21:26:00 +0100 Subject: [PATCH 11/16] Kill again, but only the main node process --- node/core/pvf/src/execute/worker.rs | 9 ++++++++- node/core/pvf/src/prepare/worker.rs | 9 ++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 7ac2c5b0dd22..2fca2d5dcca1 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -275,8 +275,15 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { gum::error!( target: LOG_TARGET, %worker_pid, - "Node and worker version mismatch, node needs restarting", + "Node and worker version mismatch, node needs restarting, forcing shutdown", ); + unsafe { + // SAFETY: Neither `getppid()` nor `kill()` never fails + let ppid = libc::getppid(); + if ppid > 1 { + libc::kill(ppid, libc::SIGKILL); + } + } return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } } diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index 8d18dd3a072e..151344dcc367 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -358,8 +358,15 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { gum::error!( target: LOG_TARGET, %worker_pid, - "Node and worker version mismatch, node needs restarting", + "Node and worker version mismatch, node needs restarting, forcing shutdown", ); + unsafe { + // SAFETY: Neither `getppid()` nor `kill()` never fails + let ppid = libc::getppid(); + if ppid > 1 { + libc::kill(ppid, libc::SIGKILL); + } + } return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } } From 1b4678b5870a8578a8ef0779fce6c133892c0a00 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Mon, 20 Mar 2023 13:15:14 +0100 Subject: [PATCH 12/16] Move unsafe code to a common safe function --- node/core/pvf/src/execute/worker.rs | 8 +------- node/core/pvf/src/lib.rs | 1 + node/core/pvf/src/prepare/worker.rs | 8 +------- node/core/pvf/src/worker_common.rs | 17 +++++++++++++++++ 4 files changed, 20 insertions(+), 14 deletions(-) diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 2fca2d5dcca1..45e11414fb02 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -277,13 +277,7 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { %worker_pid, "Node and worker version mismatch, node needs restarting, forcing shutdown", ); - unsafe { - // SAFETY: Neither `getppid()` nor `kill()` never fails - let ppid = libc::getppid(); - if ppid > 1 { - libc::kill(ppid, libc::SIGKILL); - } - } + crate::kill_node_in_emergency(); return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } } diff --git a/node/core/pvf/src/lib.rs b/node/core/pvf/src/lib.rs index 8c40bbb8b939..7a17f659c061 100644 --- a/node/core/pvf/src/lib.rs +++ b/node/core/pvf/src/lib.rs @@ -114,6 +114,7 @@ pub use pvf::PvfPrepData; pub use host::{start, Config, ValidationHost}; pub use metrics::Metrics; +pub(crate) use worker_common::kill_node_in_emergency; pub use worker_common::JOB_TIMEOUT_WALL_CLOCK_FACTOR; pub use execute::worker_entrypoint as execute_worker_entrypoint; diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index 151344dcc367..2f01e0d28d7a 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -360,13 +360,7 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { %worker_pid, "Node and worker version mismatch, node needs restarting, forcing shutdown", ); - unsafe { - // SAFETY: Neither `getppid()` nor `kill()` never fails - let ppid = libc::getppid(); - if ppid > 1 { - libc::kill(ppid, libc::SIGKILL); - } - } + crate::kill_node_in_emergency(); return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } } diff --git a/node/core/pvf/src/worker_common.rs b/node/core/pvf/src/worker_common.rs index 662e5de5303f..4f61f672174a 100644 --- a/node/core/pvf/src/worker_common.rs +++ b/node/core/pvf/src/worker_common.rs @@ -394,3 +394,20 @@ pub async fn framed_recv(r: &mut (impl AsyncRead + Unpin)) -> io::Result r.read_exact(&mut buf).await?; Ok(buf) } + +// In case of node and worker version mismatch (as a result of in-place upgrade), send `SIGKILL` +// to the node to tear it down and prevent it from raising disputes on valid candidates. Node +// restart should be handled by the node owner. As node exits, unix sockets opened to workers +// get closed by the OS and other workers receive error on socket read and also exit. Preparation +// jobs are written to the temporary files that are renamed to real artifacts on the node side, so +// no leftover artifacts are possible. +pub(crate) fn kill_node_in_emergency() { + unsafe { + // SAFETY: `getpid()` never fails but may return "no-parent" (0) or "parent-init" (1) in + // some corner cases, which is checked. `kill()` never fails. + let ppid = libc::getppid(); + if ppid > 1 { + libc::kill(ppid, libc::SIGKILL); + } + } +} From ec9c4ca0632a071da5059b5134f76fe35df101ec Mon Sep 17 00:00:00 2001 From: Marcin S Date: Mon, 20 Mar 2023 19:47:11 +0100 Subject: [PATCH 13/16] Fix libc dependency error on MacOS --- node/core/pvf/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/node/core/pvf/Cargo.toml b/node/core/pvf/Cargo.toml index 479c3ba539c5..6f86c21d13b7 100644 --- a/node/core/pvf/Cargo.toml +++ b/node/core/pvf/Cargo.toml @@ -15,6 +15,7 @@ cpu-time = "1.0.0" futures = "0.3.21" futures-timer = "3.0.2" gum = { package = "tracing-gum", path = "../../gum" } +libc = "0.2.139" pin-project = "1.0.9" rand = "0.8.5" rayon = "1.6.1" @@ -45,7 +46,6 @@ sp-tracing = { git = "https://github.com/paritytech/substrate", branch = "master substrate-build-script-utils = { git = "https://github.com/paritytech/substrate", branch = "master" } [target.'cfg(target_os = "linux")'.dependencies] -libc = "0.2.139" tikv-jemalloc-ctl = "0.5.0" [dev-dependencies] From 0bd760f53fde0363a39bec7eaa9f3dbcc07ad197 Mon Sep 17 00:00:00 2001 From: Marcin S Date: Wed, 22 Mar 2023 11:29:50 +0100 Subject: [PATCH 14/16] pvf spawning: Add some logging, add a small integration test --- node/core/pvf/src/testing.rs | 3 +++ node/core/pvf/src/worker_common.rs | 25 +++++++++++++++++++++++-- node/core/pvf/tests/it/worker_common.rs | 9 +++++++++ 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/node/core/pvf/src/testing.rs b/node/core/pvf/src/testing.rs index a4451d53e1e4..9ff6d5a15e05 100644 --- a/node/core/pvf/src/testing.rs +++ b/node/core/pvf/src/testing.rs @@ -78,6 +78,9 @@ macro_rules! decl_puppet_worker_main { let subcommand = &args[1]; match subcommand.as_ref() { + "exit" => { + std::process::exit(1); + }, "sleep" => { std::thread::sleep(std::time::Duration::from_secs(5)); }, diff --git a/node/core/pvf/src/worker_common.rs b/node/core/pvf/src/worker_common.rs index 4f61f672174a..e13e0f25c421 100644 --- a/node/core/pvf/src/worker_common.rs +++ b/node/core/pvf/src/worker_common.rs @@ -61,6 +61,8 @@ pub async fn spawn_with_program_path( gum::warn!( target: LOG_TARGET, %debug_id, + ?program_path, + ?extra_args, "cannot bind unix socket: {:?}", err, ); @@ -68,10 +70,12 @@ pub async fn spawn_with_program_path( })?; let handle = - WorkerHandle::spawn(program_path, extra_args, socket_path).map_err(|err| { + WorkerHandle::spawn(&program_path, extra_args, socket_path).map_err(|err| { gum::warn!( target: LOG_TARGET, %debug_id, + ?program_path, + ?extra_args, "cannot spawn a worker: {:?}", err, ); @@ -84,6 +88,8 @@ pub async fn spawn_with_program_path( gum::warn!( target: LOG_TARGET, %debug_id, + ?program_path, + ?extra_args, "cannot accept a worker: {:?}", err, ); @@ -92,6 +98,14 @@ pub async fn spawn_with_program_path( Ok((IdleWorker { stream, pid: handle.id() }, handle)) } _ = Delay::new(spawn_timeout).fuse() => { + gum::warn!( + target: LOG_TARGET, + %debug_id, + ?program_path, + ?extra_args, + ?spawn_timeout, + "spawning and connecting to socket timed out", + ); Err(SpawnErr::AcceptTimeout) } } @@ -162,6 +176,13 @@ where F: FnMut(Handle, UnixStream) -> Fut, Fut: futures::Future>, { + gum::debug!( + target: LOG_TARGET, + worker_pid = %std::process::id(), + "starting pvf worker ({})", + debug_id, + ); + let rt = Runtime::new().expect("Creates tokio runtime. If this panics the worker will die and the host will detect that and deal with it."); let handle = rt.handle(); let err = rt @@ -179,7 +200,7 @@ where gum::debug!( target: LOG_TARGET, worker_pid = %std::process::id(), - "pvf worker ({}): {:?}", + "quitting pvf worker ({}): {:?}", debug_id, err, ); diff --git a/node/core/pvf/tests/it/worker_common.rs b/node/core/pvf/tests/it/worker_common.rs index 7e00d005df19..72bc80916262 100644 --- a/node/core/pvf/tests/it/worker_common.rs +++ b/node/core/pvf/tests/it/worker_common.rs @@ -18,6 +18,15 @@ use crate::PUPPET_EXE; use polkadot_node_core_pvf::testing::worker_common::{spawn_with_program_path, SpawnErr}; use std::time::Duration; +// Test spawning a program that immediately exits with a failure code. +#[tokio::test] +async fn spawn_immediate_exit() { + let result = + spawn_with_program_path("integration-test", PUPPET_EXE, &["exit"], Duration::from_secs(2)) + .await; + assert!(matches!(result, Err(SpawnErr::AcceptTimeout))); +} + #[tokio::test] async fn spawn_timeout() { let result = From 0ecce2e941a8dbf629906efb3169d1098b15b198 Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 29 Mar 2023 12:09:21 +0100 Subject: [PATCH 15/16] Minor fixes --- node/core/pvf/src/execute/worker.rs | 2 +- node/core/pvf/src/lib.rs | 2 +- node/core/pvf/src/prepare/worker.rs | 2 +- node/core/pvf/src/worker_common.rs | 14 +++++++------- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 45e11414fb02..04357d8704bc 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -277,7 +277,7 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { %worker_pid, "Node and worker version mismatch, node needs restarting, forcing shutdown", ); - crate::kill_node_in_emergency(); + crate::kill_parent_node_in_emergency(); return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } } diff --git a/node/core/pvf/src/lib.rs b/node/core/pvf/src/lib.rs index 7a17f659c061..88134529bc4b 100644 --- a/node/core/pvf/src/lib.rs +++ b/node/core/pvf/src/lib.rs @@ -114,7 +114,7 @@ pub use pvf::PvfPrepData; pub use host::{start, Config, ValidationHost}; pub use metrics::Metrics; -pub(crate) use worker_common::kill_node_in_emergency; +pub(crate) use worker_common::kill_parent_node_in_emergency; pub use worker_common::JOB_TIMEOUT_WALL_CLOCK_FACTOR; pub use execute::worker_entrypoint as execute_worker_entrypoint; diff --git a/node/core/pvf/src/prepare/worker.rs b/node/core/pvf/src/prepare/worker.rs index 2f01e0d28d7a..1ccba603c1fb 100644 --- a/node/core/pvf/src/prepare/worker.rs +++ b/node/core/pvf/src/prepare/worker.rs @@ -360,7 +360,7 @@ pub fn worker_entrypoint(socket_path: &str, node_version: Option<&str>) { %worker_pid, "Node and worker version mismatch, node needs restarting, forcing shutdown", ); - crate::kill_node_in_emergency(); + crate::kill_parent_node_in_emergency(); return Err(io::Error::new(io::ErrorKind::Unsupported, "Version mismatch")) } } diff --git a/node/core/pvf/src/worker_common.rs b/node/core/pvf/src/worker_common.rs index e13e0f25c421..3ed2994a2f94 100644 --- a/node/core/pvf/src/worker_common.rs +++ b/node/core/pvf/src/worker_common.rs @@ -416,13 +416,13 @@ pub async fn framed_recv(r: &mut (impl AsyncRead + Unpin)) -> io::Result Ok(buf) } -// In case of node and worker version mismatch (as a result of in-place upgrade), send `SIGKILL` -// to the node to tear it down and prevent it from raising disputes on valid candidates. Node -// restart should be handled by the node owner. As node exits, unix sockets opened to workers -// get closed by the OS and other workers receive error on socket read and also exit. Preparation -// jobs are written to the temporary files that are renamed to real artifacts on the node side, so -// no leftover artifacts are possible. -pub(crate) fn kill_node_in_emergency() { +/// In case of node and worker version mismatch (as a result of in-place upgrade), send `SIGKILL` +/// to the node to tear it down and prevent it from raising disputes on valid candidates. Node +/// restart should be handled by the node owner. As node exits, unix sockets opened to workers +/// get closed by the OS and other workers receive error on socket read and also exit. Preparation +/// jobs are written to the temporary files that are renamed to real artifacts on the node side, so +/// no leftover artifacts are possible. +pub(crate) fn kill_parent_node_in_emergency() { unsafe { // SAFETY: `getpid()` never fails but may return "no-parent" (0) or "parent-init" (1) in // some corner cases, which is checked. `kill()` never fails. From 456314c553b5bc937f76070affa7bc4adba20e5f Mon Sep 17 00:00:00 2001 From: Dmitry Sinyavin Date: Wed, 29 Mar 2023 21:52:14 +0100 Subject: [PATCH 16/16] Restart CI