From a5711fb3bedcd506ae089fb123e9c69cdb8b6e59 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Mon, 4 Mar 2024 14:32:48 -0800 Subject: [PATCH 01/35] first steps for lookahead collator implementation --- Cargo.lock | 5 + Cargo.toml | 2 + client/consensus/Cargo.toml | 2 + client/consensus/src/collators.rs | 1 + client/consensus/src/collators/lookahead.rs | 520 ++++++++++++++++++ .../templates/frontier/runtime/Cargo.toml | 2 + .../templates/frontier/runtime/src/lib.rs | 9 + .../templates/simple/runtime/Cargo.toml | 2 + .../templates/simple/runtime/src/lib.rs | 9 + runtime/dancebox/Cargo.toml | 2 + runtime/dancebox/src/lib.rs | 9 + 11 files changed, 563 insertions(+) create mode 100644 client/consensus/src/collators/lookahead.rs diff --git a/Cargo.lock b/Cargo.lock index cbfc772db..ee991b40c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1759,6 +1759,7 @@ dependencies = [ name = "container-chain-template-frontier-runtime" version = "0.1.0" dependencies = [ + "async-backing-primitives", "ccp-xcm", "cumulus-pallet-dmp-queue", "cumulus-pallet-parachain-system", @@ -1931,6 +1932,7 @@ dependencies = [ name = "container-chain-template-simple-runtime" version = "0.1.0" dependencies = [ + "async-backing-primitives", "cumulus-pallet-dmp-queue", "cumulus-pallet-parachain-system", "cumulus-pallet-session-benchmarking", @@ -2986,6 +2988,7 @@ name = "dancebox-runtime" version = "0.1.0" dependencies = [ "assets-common", + "async-backing-primitives", "container-chain-template-frontier-runtime", "container-chain-template-simple-runtime", "cumulus-pallet-dmp-queue", @@ -15332,6 +15335,7 @@ checksum = "69758bda2e78f098e4ccb393021a0963bb3442eac05f135c30f61b7370bbafae" name = "tc-consensus" version = "0.1.0" dependencies = [ + "async-backing-primitives", "async-trait", "cumulus-client-collator", "cumulus-client-consensus-aura", @@ -15352,6 +15356,7 @@ dependencies = [ "parking_lot 0.12.1", "polkadot-core-primitives", "polkadot-node-primitives", + "polkadot-node-subsystem", "polkadot-overseer", "polkadot-parachain-primitives", "polkadot-primitives", diff --git a/Cargo.toml b/Cargo.toml index 5584c4654..4e6d3cfc7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -68,6 +68,7 @@ test-relay-sproof-builder = { git = "https://github.com/moondance-labs/dancekit" dc-orchestrator-chain-interface = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0" } # Moonkit (wasm) +async-backing-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } nimbus-consensus = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0" } nimbus-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } pallet-async-backing = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } @@ -192,6 +193,7 @@ westend-runtime-constants = { git = "https://github.com/moondance-labs/polkadot- # Polkadot (client) polkadot-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +polkadot-node-subsystem = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } polkadot-overseer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } polkadot-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } polkadot-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } diff --git a/client/consensus/Cargo.toml b/client/consensus/Cargo.toml index f242b0bfa..86e596476 100644 --- a/client/consensus/Cargo.toml +++ b/client/consensus/Cargo.toml @@ -43,10 +43,12 @@ cumulus-relay-chain-interface = { workspace = true } # Polkadot polkadot-node-primitives = { workspace = true } +polkadot-node-subsystem = { workspace = true } polkadot-overseer = { workspace = true } polkadot-primitives = { workspace = true } # Nimbus Dependencies +async-backing-primitives = { workspace = true } nimbus-consensus = { workspace = true } nimbus-primitives = { workspace = true, features = [ "std" ] } diff --git a/client/consensus/src/collators.rs b/client/consensus/src/collators.rs index fc5e0a8b7..0123263f4 100644 --- a/client/consensus/src/collators.rs +++ b/client/consensus/src/collators.rs @@ -15,6 +15,7 @@ // along with Tanssi. If not, see . pub mod basic; +pub mod lookahead; use { crate::{find_pre_digest, AuthorityId, OrchestratorAuraWorkerAuxData}, diff --git a/client/consensus/src/collators/lookahead.rs b/client/consensus/src/collators/lookahead.rs new file mode 100644 index 000000000..4644bd61d --- /dev/null +++ b/client/consensus/src/collators/lookahead.rs @@ -0,0 +1,520 @@ +// Copyright (C) Moondance Labs Ltd. +// This file is part of Tanssi. + +// Tanssi 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. + +// Tanssi 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 Tanssi. If not, see . + +//! A collator for Tanssi Aura that looks ahead of the most recently included parachain block +//! when determining what to build upon. +//! +//! This collator also builds additional blocks when the maximum backlog is not saturated. +//! The size of the backlog is determined by invoking a runtime API. If that runtime API +//! is not supported, this assumes a maximum backlog size of 1. +//! +//! This takes more advantage of asynchronous backing, though not complete advantage. +//! When the backlog is not saturated, this approach lets the backlog temporarily 'catch up' +//! with periods of higher throughput. When the backlog is saturated, we typically +//! fall back to the limited cadence of a single parachain block per relay-chain block. +//! +//! Despite this, the fact that there is a backlog at all allows us to spend more time +//! building the block, as there is some buffer before it can get posted to the relay-chain. +//! The main limitation is block propagation time - i.e. the new blocks created by an author +//! must be propagated to the next author before their turn. + +use async_backing_primitives::UnincludedSegmentApi; +use cumulus_client_collator::service::ServiceInterface as CollatorServiceInterface; +use cumulus_client_consensus_common::{ + self as consensus_common, load_abridged_host_configuration, ParachainBlockImportMarker, + ParentSearchParams, +}; +use cumulus_client_consensus_proposer::ProposerInterface; +use cumulus_primitives_core::{ + relay_chain::Hash as PHash, CollectCollationInfo, PersistedValidationData, +}; +use cumulus_relay_chain_interface::RelayChainInterface; +use parity_scale_codec::{Codec, Encode}; +use polkadot_node_primitives::SubmitCollationParams; +use polkadot_node_subsystem::messages::{ + CollationGenerationMessage, RuntimeApiMessage, RuntimeApiRequest, +}; +use polkadot_overseer::Handle as OverseerHandle; +use polkadot_primitives::{CollatorPair, Id as ParaId, OccupiedCoreAssumption}; + +use futures::{channel::oneshot, prelude::*}; +use sc_client_api::{backend::AuxStore, BlockBackend, BlockOf}; +use sc_consensus::BlockImport; +use sc_consensus_slots::InherentDataProviderExt; +use sp_api::ProvideRuntimeApi; +use sp_application_crypto::AppPublic; +use sp_blockchain::HeaderBackend; +use sp_consensus::SyncOracle; +use sp_consensus_aura::{Slot, SlotDuration}; +use sp_core::crypto::Pair; +use sp_inherents::CreateInherentDataProviders; +use sp_keystore::KeystorePtr; +use sp_runtime::traits::{Block as BlockT, Header as HeaderT, Member}; +//use sp_timestamp::Timestamp; +use std::{convert::TryFrom, error::Error, sync::Arc, time::Duration}; + +use crate::{ + collators::{self as collator_util, SlotClaim, tanssi_claim_slot}, consensus_orchestrator::RetrieveAuthoritiesFromOrchestrator, + OrchestratorAuraWorkerAuxData, +}; + +/// Parameters for [`run`]. +pub struct Params { + pub create_inherent_data_providers: CIDP, + pub get_orchestrator_aux_data: GOH, + pub block_import: BI, + pub para_client: Arc, + pub para_backend: Arc, + pub relay_client: RClient, + pub code_hash_provider: CHP, + pub sync_oracle: SO, + pub keystore: KeystorePtr, + pub collator_key: CollatorPair, + pub para_id: ParaId, + pub overseer_handle: OverseerHandle, + pub slot_duration: SlotDuration, + pub relay_chain_slot_duration: Duration, + pub proposer: Proposer, + pub collator_service: CS, + pub authoring_duration: Duration, + pub force_authoring: bool, +} + +/// Run async-backing-friendly for Tanssi Aura. +pub fn run( + mut params: Params, +) -> impl Future + Send + 'static +where + Block: BlockT, + Client: ProvideRuntimeApi + + BlockOf + + AuxStore + + HeaderBackend + + BlockBackend + + Send + + Sync + + 'static, + Client::Api: UnincludedSegmentApi, + //AuraApi + CollectCollationInfo, //+ AuraUnincludedSegmentApi, + Backend: sc_client_api::Backend + 'static, + RClient: RelayChainInterface + Clone + 'static, + CIDP: CreateInherentDataProviders + + Send + + 'static + + Clone, + CIDP::InherentDataProviders: Send + InherentDataProviderExt, + BI: BlockImport + ParachainBlockImportMarker + Send + Sync + 'static, + SO: SyncOracle + Send + Sync + Clone + 'static, + Proposer: ProposerInterface + Send + Sync + 'static, + CS: CollatorServiceInterface + Send + Sync + 'static, + CHP: consensus_common::ValidationCodeHashProvider + Send + 'static, + P: Pair + Sync + Send + 'static, + P::Public: AppPublic + Member + Codec, + P::Signature: TryFrom> + Member + Codec, + GOH: RetrieveAuthoritiesFromOrchestrator< + Block, + (PHash, PersistedValidationData), + OrchestratorAuraWorkerAuxData

, + > + + 'static + + Sync + + Send, +{ + // This is an arbitrary value which is likely guaranteed to exceed any reasonable + // limit, as it would correspond to 10 non-included blocks. + // + // Since we only search for parent blocks which have already been imported, + // we can guarantee that all imported blocks respect the unincluded segment + // rules specified by the parachain's runtime and thus will never be too deep. + const PARENT_SEARCH_DEPTH: usize = 10; + + log::info!("LOOKAHEAD COLLATOR RUNNING..."); + + async move { + cumulus_client_collator::initialize_collator_subsystems( + &mut params.overseer_handle, + params.collator_key, + params.para_id, + ) + .await; + + let mut import_notifications = match params.relay_client.import_notification_stream().await + { + Ok(s) => s, + Err(err) => { + tracing::error!( + target: crate::LOG_TARGET, + ?err, + "Failed to initialize consensus: no relay chain import notification stream" + ); + + return + }, + }; + + let mut collator = { + let params = collator_util::Params { + create_inherent_data_providers: params.create_inherent_data_providers.clone(), + block_import: params.block_import, + relay_client: params.relay_client.clone(), + keystore: params.keystore.clone(), + para_id: params.para_id, + proposer: params.proposer, + collator_service: params.collator_service, + }; + + collator_util::Collator::::new(params) + }; + + while let Some(relay_parent_header) = import_notifications.next().await { + let relay_parent = relay_parent_header.hash(); + + if !is_para_scheduled(relay_parent, params.para_id, &mut params.overseer_handle).await { + tracing::trace!( + target: crate::LOG_TARGET, + ?relay_parent, + ?params.para_id, + "Para is not scheduled on any core, skipping import notification", + ); + + continue + } + + let max_pov_size = match params + .relay_client + .persisted_validation_data( + relay_parent, + params.para_id, + OccupiedCoreAssumption::Included, + ) + .await + { + Ok(None) => continue, + Ok(Some(pvd)) => pvd.max_pov_size, + Err(err) => { + tracing::error!(target: crate::LOG_TARGET, ?err, "Failed to gather information from relay-client"); + continue + }, + }; + + let (slot_now, _) = match consensus_common::relay_slot_and_timestamp( + &relay_parent_header, + params.relay_chain_slot_duration, + ) { + None => continue, + Some((relay_slot, relay_timestamp)) => { + let our_slot = Slot::from_timestamp(relay_timestamp, params.slot_duration); + tracing::debug!( + target: crate::LOG_TARGET, + relay_slot = ?relay_slot, + para_slot = ?our_slot, + timestamp = ?relay_timestamp, + slot_duration = ?params.slot_duration, + relay_chain_slot_duration = ?params.relay_chain_slot_duration, + "Adjusted relay-chain slot to parachain slot" + ); + (our_slot, relay_timestamp) + }, + }; + + let parent_search_params = ParentSearchParams { + relay_parent, + para_id: params.para_id, + ancestry_lookback: max_ancestry_lookback(relay_parent, ¶ms.relay_client).await, + max_depth: PARENT_SEARCH_DEPTH, + ignore_alternative_branches: true, + }; + + let potential_parents = + cumulus_client_consensus_common::find_potential_parents::( + parent_search_params, + &*params.para_backend, + ¶ms.relay_client, + ) + .await; + + let mut potential_parents = match potential_parents { + Err(e) => { + tracing::error!( + target: crate::LOG_TARGET, + ?relay_parent, + err = ?e, + "Could not fetch potential parents to build upon" + ); + + continue + }, + Ok(x) => x, + }; + + let included_block = match potential_parents.iter().find(|x| x.depth == 0) { + None => continue, // also serves as an `is_empty` check. + Some(b) => b.hash, + }; + + let para_client = &*params.para_client; + let keystore = ¶ms.keystore; + let can_build_upon = |block_hash, aux_data| { + can_build_upon::<_, _, P>( + slot_now, + aux_data, + block_hash, + included_block, + params.force_authoring, + para_client, + &keystore, + ) + }; + + // Sort by depth, ascending, to choose the longest chain. + // + // If the longest chain has space, build upon that. Otherwise, don't + // build at all. + potential_parents.sort_by_key(|a| a.depth); + let initial_parent = match potential_parents.pop() { + None => continue, + Some(p) => p, + }; + + // Build in a loop until not allowed. Note that the authorities can change + // at any block, so we need to re-claim our slot every time. + let mut parent_hash = initial_parent.hash; + let mut parent_header = initial_parent.header; + let overseer_handle = &mut params.overseer_handle; + + // This needs to change to support elastic scaling, but for continuously + // scheduled chains this ensures that the backlog will grow steadily. + for n_built in 0..2 { + let validation_data = PersistedValidationData { + parent_head: parent_header.encode().into(), + relay_parent_number: *relay_parent_header.number(), + relay_parent_storage_root: *relay_parent_header.state_root(), + max_pov_size, + }; + + // Retrieve authorities that are able to produce the block + let aux_data = match params + .get_orchestrator_aux_data + .retrieve_authorities_from_orchestrator( + parent_hash, + (relay_parent_header.hash(), validation_data.clone()), + ) + .await + { + Err(e) => { + tracing::error!(target: crate::LOG_TARGET, ?e); + break; + }, + Ok(h) => h, + }; + + let mut slot_claim = match can_build_upon(parent_header.clone(), aux_data).await { + Ok(None) => break, + Err(e) => { + tracing::error!(target: crate::LOG_TARGET, ?e); + break; + }, + Ok(Some(c)) => c, + }; + + tracing::debug!( + target: crate::LOG_TARGET, + ?relay_parent, + unincluded_segment_len = initial_parent.depth + n_built, + "Slot claimed. Building" + ); + + // Build and announce collations recursively until + // `can_build_upon` fails or building a collation fails. + let (parachain_inherent_data, other_inherent_data) = match collator + .create_inherent_data( + relay_parent, + &validation_data, + parent_hash, + None, + ) + .await + { + Err(err) => { + tracing::error!(target: crate::LOG_TARGET, ?err); + break + }, + Ok(x) => x, + }; + + let validation_code_hash = match params.code_hash_provider.code_hash_at(parent_hash) + { + None => { + tracing::error!(target: crate::LOG_TARGET, ?parent_hash, "Could not fetch validation code hash"); + break + }, + Some(v) => v, + }; + + match collator + .collate( + &parent_header, + &mut slot_claim, + None, + (parachain_inherent_data, other_inherent_data), + params.authoring_duration, + // Set the block limit to 50% of the maximum PoV size. + // + // TODO: If we got benchmarking that includes the proof size, + // we should be able to use the maximum pov size. + (validation_data.max_pov_size / 2) as usize, + ) + .await + { + Ok(Some((collation, block_data, new_block_hash))) => { + // Here we are assuming that the import logic protects against equivocations + // and provides sybil-resistance, as it should. + collator.collator_service().announce_block(new_block_hash, None); + + // Send a submit-collation message to the collation generation subsystem, + // which then distributes this to validators. + // + // Here we are assuming that the leaf is imported, as we've gotten an + // import notification. + overseer_handle + .send_msg( + CollationGenerationMessage::SubmitCollation( + SubmitCollationParams { + relay_parent, + collation, + parent_head: parent_header.encode().into(), + validation_code_hash, + result_sender: None, + }, + ), + "SubmitCollation", + ) + .await; + + parent_hash = new_block_hash; + parent_header = block_data.into_header(); + }, + Ok(None) => { + tracing::debug!(target: crate::LOG_TARGET, "Lookahead collator: No block proposal"); + } + Err(err) => { + tracing::error!(target: crate::LOG_TARGET, ?err); + break + }, + } + } + } + } +} + +// Checks if we own the slot at the given block and whether there +// is space in the unincluded segment. +async fn can_build_upon( + slot: Slot, + aux_data: OrchestratorAuraWorkerAuxData

, + parent_header: Block::Header, + included_block: Block::Hash, + force_authoring: bool, + client: &Client, + keystore: &KeystorePtr, +) -> Result>, Box> +where + Client: ProvideRuntimeApi, + Client::Api: UnincludedSegmentApi,//AuraApi, //+ AuraUnincludedSegmentApi, + P: Pair + Send + Sync + 'static, + P::Public: Codec + std::fmt::Debug, + P::Signature: Codec, +{ + let runtime_api = client.runtime_api(); + let slot_claim = tanssi_claim_slot::(aux_data, &parent_header, slot, force_authoring, keystore); + + // Here we lean on the property that building on an empty unincluded segment must always + // be legal. Skipping the runtime API query here allows us to seamlessly run this + // collator against chains which have not yet upgraded their runtime. + if parent_header.hash() != included_block { + if !runtime_api.can_build_upon(parent_header.hash(), included_block, slot)? { + return Ok(None) + } + } + + slot_claim +} + +/// Reads allowed ancestry length parameter from the relay chain storage at the given relay parent. +/// +/// Falls back to 0 in case of an error. +async fn max_ancestry_lookback( + relay_parent: PHash, + relay_client: &impl RelayChainInterface, +) -> usize { + match load_abridged_host_configuration(relay_parent, relay_client).await { + Ok(Some(config)) => config.async_backing_params.allowed_ancestry_len as usize, + Ok(None) => { + tracing::error!( + target: crate::LOG_TARGET, + "Active config is missing in relay chain storage", + ); + 0 + }, + Err(err) => { + tracing::error!( + target: crate::LOG_TARGET, + ?err, + ?relay_parent, + "Failed to read active config from relay chain client", + ); + 0 + }, + } +} + +// Checks if there exists a scheduled core for the para at the provided relay parent. +// +// Falls back to `false` in case of an error. +async fn is_para_scheduled( + relay_parent: PHash, + para_id: ParaId, + overseer_handle: &mut OverseerHandle, +) -> bool { + let (tx, rx) = oneshot::channel(); + let request = RuntimeApiRequest::AvailabilityCores(tx); + overseer_handle + .send_msg(RuntimeApiMessage::Request(relay_parent, request), "LookaheadCollator") + .await; + + let cores = match rx.await { + Ok(Ok(cores)) => cores, + Ok(Err(error)) => { + tracing::error!( + target: crate::LOG_TARGET, + ?error, + ?relay_parent, + "Failed to query availability cores runtime API", + ); + return false + }, + Err(oneshot::Canceled) => { + tracing::error!( + target: crate::LOG_TARGET, + ?relay_parent, + "Sender for availability cores runtime request dropped", + ); + return false + }, + }; + + cores.iter().any(|core| core.para_id() == Some(para_id)) +} \ No newline at end of file diff --git a/container-chains/templates/frontier/runtime/Cargo.toml b/container-chains/templates/frontier/runtime/Cargo.toml index 764038355..496bd1f7f 100644 --- a/container-chains/templates/frontier/runtime/Cargo.toml +++ b/container-chains/templates/frontier/runtime/Cargo.toml @@ -26,6 +26,7 @@ tp-consensus = { workspace = true } tp-impl-tanssi-pallets-config = { workspace = true } # Moonkit +async-backing-primitives = { workspace = true } nimbus-primitives = { workspace = true } pallet-async-backing = { workspace = true } pallet-author-inherent = { workspace = true } @@ -124,6 +125,7 @@ substrate-wasm-builder = { workspace = true } [features] default = [ "std" ] std = [ + "async-backing-primitives/std", "ccp-xcm/std", "cumulus-pallet-dmp-queue/std", "cumulus-pallet-parachain-system/std", diff --git a/container-chains/templates/frontier/runtime/src/lib.rs b/container-chains/templates/frontier/runtime/src/lib.rs index 0ba300708..dfa82e368 100644 --- a/container-chains/templates/frontier/runtime/src/lib.rs +++ b/container-chains/templates/frontier/runtime/src/lib.rs @@ -1075,6 +1075,15 @@ impl_runtime_apis! { } } + impl async_backing_primitives::UnincludedSegmentApi for Runtime { + fn can_build_upon( + included_hash: ::Hash, + slot: async_backing_primitives::Slot, + ) -> bool { + ConsensusHook::can_build_upon(included_hash, slot) + } + } + impl sp_genesis_builder::GenesisBuilder for Runtime { fn create_default_config() -> Vec { create_default_config::() diff --git a/container-chains/templates/simple/runtime/Cargo.toml b/container-chains/templates/simple/runtime/Cargo.toml index 066fb9991..8c68ab5b3 100644 --- a/container-chains/templates/simple/runtime/Cargo.toml +++ b/container-chains/templates/simple/runtime/Cargo.toml @@ -24,6 +24,7 @@ tp-consensus = { workspace = true } tp-impl-tanssi-pallets-config = { workspace = true } # Moonkit +async-backing-primitives = { workspace = true } nimbus-primitives = { workspace = true } pallet-async-backing = { workspace = true } pallet-author-inherent = { workspace = true } @@ -102,6 +103,7 @@ substrate-wasm-builder = { workspace = true } [features] default = [ "std" ] std = [ + "async-backing-primitives/std", "cumulus-pallet-dmp-queue/std", "cumulus-pallet-parachain-system/std", "cumulus-pallet-session-benchmarking/std", diff --git a/container-chains/templates/simple/runtime/src/lib.rs b/container-chains/templates/simple/runtime/src/lib.rs index fbc7fb8a3..93535b17b 100644 --- a/container-chains/templates/simple/runtime/src/lib.rs +++ b/container-chains/templates/simple/runtime/src/lib.rs @@ -799,6 +799,15 @@ impl_runtime_apis! { } } + impl async_backing_primitives::UnincludedSegmentApi for Runtime { + fn can_build_upon( + included_hash: ::Hash, + slot: async_backing_primitives::Slot, + ) -> bool { + ConsensusHook::can_build_upon(included_hash, slot) + } + } + impl sp_genesis_builder::GenesisBuilder for Runtime { fn create_default_config() -> Vec { create_default_config::() diff --git a/runtime/dancebox/Cargo.toml b/runtime/dancebox/Cargo.toml index ad8a51080..c4ab5154d 100644 --- a/runtime/dancebox/Cargo.toml +++ b/runtime/dancebox/Cargo.toml @@ -40,6 +40,7 @@ pallet-stream-payment = { workspace = true } runtime-common = { workspace = true } # Moonkit +async-backing-primitives = { workspace = true } nimbus-primitives = { workspace = true } pallet-async-backing = { workspace = true } pallet-author-inherent = { workspace = true } @@ -146,6 +147,7 @@ default = [ "std", ] std = [ + "async-backing-primitives/std", "container-chain-template-frontier-runtime/std", "container-chain-template-simple-runtime/std", "cumulus-pallet-dmp-queue/std", diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index dd69ff8fe..40843d410 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -1798,6 +1798,15 @@ impl_runtime_apis! { } } + impl async_backing_primitives::UnincludedSegmentApi for Runtime { + fn can_build_upon( + included_hash: ::Hash, + slot: async_backing_primitives::Slot, + ) -> bool { + ConsensusHook::can_build_upon(included_hash, slot) + } + } + impl sp_genesis_builder::GenesisBuilder for Runtime { fn create_default_config() -> Vec { create_default_config::() From 7e41cd92bc1fa2b78ff92ed1bd248882cdead17e Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 5 Mar 2024 08:14:09 -0800 Subject: [PATCH 02/35] experiment lookahead collator with current 12s --- Cargo.lock | 1 + client/consensus/src/collators/lookahead.rs | 39 ++++++++--------- .../templates/frontier/runtime/src/lib.rs | 7 +-- .../templates/simple/runtime/src/lib.rs | 7 +-- node/src/service.rs | 43 ++++++++++++++++--- primitives/consensus/Cargo.toml | 2 + primitives/consensus/src/lib.rs | 6 +++ runtime/dancebox/src/lib.rs | 6 +-- 8 files changed, 74 insertions(+), 37 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ee991b40c..208f36c41 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -15830,6 +15830,7 @@ dependencies = [ "cumulus-primitives-core", "frame-support", "frame-system", + "log", "nimbus-primitives", "parity-scale-codec", "scale-info", diff --git a/client/consensus/src/collators/lookahead.rs b/client/consensus/src/collators/lookahead.rs index 4644bd61d..dc7ca3a38 100644 --- a/client/consensus/src/collators/lookahead.rs +++ b/client/consensus/src/collators/lookahead.rs @@ -210,26 +210,6 @@ where }, }; - let (slot_now, _) = match consensus_common::relay_slot_and_timestamp( - &relay_parent_header, - params.relay_chain_slot_duration, - ) { - None => continue, - Some((relay_slot, relay_timestamp)) => { - let our_slot = Slot::from_timestamp(relay_timestamp, params.slot_duration); - tracing::debug!( - target: crate::LOG_TARGET, - relay_slot = ?relay_slot, - para_slot = ?our_slot, - timestamp = ?relay_timestamp, - slot_duration = ?params.slot_duration, - relay_chain_slot_duration = ?params.relay_chain_slot_duration, - "Adjusted relay-chain slot to parachain slot" - ); - (our_slot, relay_timestamp) - }, - }; - let parent_search_params = ParentSearchParams { relay_parent, para_id: params.para_id, @@ -267,7 +247,7 @@ where let para_client = &*params.para_client; let keystore = ¶ms.keystore; - let can_build_upon = |block_hash, aux_data| { + let can_build_upon = |slot_now, block_hash, aux_data| { can_build_upon::<_, _, P>( slot_now, aux_data, @@ -321,7 +301,22 @@ where Ok(h) => h, }; - let mut slot_claim = match can_build_upon(parent_header.clone(), aux_data).await { + let inherent_providers = match params + .create_inherent_data_providers + .create_inherent_data_providers( + parent_hash, + (relay_parent_header.hash(), validation_data.clone()), + ) + .await + { + Err(e) => { + tracing::error!(target: crate::LOG_TARGET, ?e); + break; + }, + Ok(h) => h, + }; + + let mut slot_claim = match can_build_upon(inherent_providers.slot(), parent_header.clone(), aux_data).await { Ok(None) => break, Err(e) => { tracing::error!(target: crate::LOG_TARGET, ?e); diff --git a/container-chains/templates/frontier/runtime/src/lib.rs b/container-chains/templates/frontier/runtime/src/lib.rs index dfa82e368..9c467f47e 100644 --- a/container-chains/templates/frontier/runtime/src/lib.rs +++ b/container-chains/templates/frontier/runtime/src/lib.rs @@ -22,6 +22,7 @@ #[cfg(feature = "std")] include!(concat!(env!("OUT_DIR"), "/wasm_binary.rs")); +use cumulus_pallet_parachain_system::RelayNumberMonotonicallyIncreases; #[cfg(feature = "std")] use sp_version::NativeVersion; @@ -511,7 +512,7 @@ parameter_types! { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 2; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 1; pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< @@ -530,7 +531,7 @@ impl cumulus_pallet_parachain_system::Config for Runtime { type ReservedDmpWeight = ReservedDmpWeight; type XcmpMessageHandler = XcmpQueue; type ReservedXcmpWeight = ReservedXcmpWeight; - type CheckAssociatedRelayNumber = RelayNumberStrictlyIncreases; + type CheckAssociatedRelayNumber = RelayNumberMonotonicallyIncreases; type ConsensusHook = ConsensusHook; } @@ -543,7 +544,7 @@ impl Get<(Slot, SlotDuration)> for ParaSlotProvider { } impl pallet_async_backing::Config for Runtime { - type AllowMultipleBlocksPerSlot = ConstBool; + type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; } diff --git a/container-chains/templates/simple/runtime/src/lib.rs b/container-chains/templates/simple/runtime/src/lib.rs index 93535b17b..65443c293 100644 --- a/container-chains/templates/simple/runtime/src/lib.rs +++ b/container-chains/templates/simple/runtime/src/lib.rs @@ -22,6 +22,7 @@ #[cfg(feature = "std")] include!(concat!(env!("OUT_DIR"), "/wasm_binary.rs")); +use cumulus_pallet_parachain_system::RelayNumberMonotonicallyIncreases; #[cfg(feature = "std")] use sp_version::NativeVersion; @@ -401,7 +402,7 @@ parameter_types! { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 2; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 1; pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< @@ -420,7 +421,7 @@ impl cumulus_pallet_parachain_system::Config for Runtime { type ReservedDmpWeight = ReservedDmpWeight; type XcmpMessageHandler = XcmpQueue; type ReservedXcmpWeight = ReservedXcmpWeight; - type CheckAssociatedRelayNumber = RelayNumberStrictlyIncreases; + type CheckAssociatedRelayNumber = RelayNumberMonotonicallyIncreases; type ConsensusHook = ConsensusHook; } @@ -433,7 +434,7 @@ impl sp_core::Get<(Slot, SlotDuration)> for ParaSlotProvider { } impl pallet_async_backing::Config for Runtime { - type AllowMultipleBlocksPerSlot = ConstBool; + type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; } diff --git a/node/src/service.rs b/node/src/service.rs index ce5cff6d7..3488c716a 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -73,6 +73,7 @@ use { std::{sync::Arc, time::Duration}, substrate_prometheus_endpoint::Registry, tc_consensus::{ + collators::lookahead::{self as lookahead_tanssi_aura, Params as LookaheadTanssiAuraParams}, collators::basic::{self as basic_tanssi_aura, Params as BasicTanssiAuraParams}, OrchestratorAuraWorkerAuxData, }, @@ -421,6 +422,7 @@ async fn start_node_impl( let node_keystore = node_builder.keystore_container.keystore().clone(); let node_telemetry_handle = node_builder.telemetry.as_ref().map(|t| t.handle()).clone(); let node_client = node_builder.client.clone(); + let node_backend = node_builder.backend.clone(); let relay_interface = relay_chain_interface.clone(); let node_sync_service = node_builder.network.sync_service.clone(); let overseer = overseer_handle.clone(); @@ -435,6 +437,7 @@ async fn start_node_impl( move || { start_consensus_orchestrator( node_client.clone(), + node_backend.clone(), block_import.clone(), node_spawn_handle.clone(), relay_interface.clone(), @@ -622,8 +625,10 @@ pub async fn start_node_impl_container( let node_spawn_handle = node_builder.task_manager.spawn_handle().clone(); let node_client = node_builder.client.clone(); + let node_backend = node_builder.backend.clone(); start_consensus_container( node_client.clone(), + node_backend.clone(), orchestrator_client.clone(), block_import.clone(), prometheus_registry.clone(), @@ -671,6 +676,7 @@ fn build_manual_seal_import_queue( #[sc_tracing::logging::prefix_logs_with(container_log_str(para_id))] fn start_consensus_container( client: Arc, + backend: Arc, orchestrator_client: Arc, block_import: ContainerChainBlockImport, prometheus_registry: Option, @@ -713,7 +719,17 @@ fn start_consensus_container( let relay_chain_interace_for_orch = relay_chain_interface.clone(); let orchestrator_client_for_cidp = orchestrator_client; - let params = BasicTanssiAuraParams { + let client_for_hash_provider = client.clone(); + + let code_hash_provider = move |block_hash| { + client_for_hash_provider + .code_at(block_hash) + .ok() + .map(polkadot_primitives::ValidationCode) + .map(|c| c.hash()) + }; + + let params = LookaheadTanssiAuraParams { create_inherent_data_providers: move |_block_hash, (relay_parent, _validation_data)| { let relay_chain_interface = relay_chain_interace_for_cidp.clone(); let orchestrator_chain_interface = orchestrator_chain_interface.clone(); @@ -812,15 +828,18 @@ fn start_consensus_container( collator_service, // Very limited proposal time. authoring_duration: Duration::from_millis(500), - collation_request_receiver: None, + para_backend: backend, + code_hash_provider, + //collation_request_receiver: None, }; - let fut = basic_tanssi_aura::run::(params); + let fut = lookahead_tanssi_aura::run::(params); spawner.spawn("tanssi-aura-container", None, fut); } fn start_consensus_orchestrator( client: Arc, + backend: Arc, block_import: ParachainBlockImport, spawner: SpawnTaskHandle, relay_chain_interface: Arc, @@ -849,8 +868,17 @@ fn start_consensus_orchestrator( let relay_chain_interace_for_cidp = relay_chain_interface.clone(); let client_set_aside_for_cidp = client.clone(); let client_set_aside_for_orch = client.clone(); + let client_for_hash_provider = client.clone(); + + let code_hash_provider = move |block_hash| { + client_for_hash_provider + .code_at(block_hash) + .ok() + .map(polkadot_primitives::ValidationCode) + .map(|c| c.hash()) + }; - let params = BasicTanssiAuraParams { + let params = LookaheadTanssiAuraParams { create_inherent_data_providers: move |block_hash, (relay_parent, _validation_data)| { let relay_chain_interface = relay_chain_interace_for_cidp.clone(); let client_set_aside_for_cidp = client_set_aside_for_cidp.clone(); @@ -930,10 +958,13 @@ fn start_consensus_orchestrator( collator_service, // Very limited proposal time. authoring_duration: Duration::from_millis(500), - collation_request_receiver: None, + code_hash_provider, + para_backend: backend, + //collation_request_receiver: None, }; - let fut = basic_tanssi_aura::run::(params); + //let fut = basic_tanssi_aura::run::(params); + let fut = lookahead_tanssi_aura::run::(params); spawner.spawn("tanssi-aura", None, fut); } diff --git a/primitives/consensus/Cargo.toml b/primitives/consensus/Cargo.toml index 7a5cbae46..19819c35a 100644 --- a/primitives/consensus/Cargo.toml +++ b/primitives/consensus/Cargo.toml @@ -8,6 +8,7 @@ version = "0.1.0" [package.metadata.docs.rs] targets = [ "x86_64-unknown-linux-gnu" ] [dependencies] +log = { workspace = true } cumulus-primitives-core = { workspace = true } frame-support = { workspace = true } frame-system = { workspace = true } @@ -33,6 +34,7 @@ std = [ "cumulus-primitives-core/std", "frame-support/std", "frame-system/std", + "log/std", "nimbus-primitives/std", "parity-scale-codec/std", "scale-info/std", diff --git a/primitives/consensus/src/lib.rs b/primitives/consensus/src/lib.rs index de8ed4709..a3fa923ce 100644 --- a/primitives/consensus/src/lib.rs +++ b/primitives/consensus/src/lib.rs @@ -58,8 +58,14 @@ where "Slot duration cannot be zero." ); + log::info!("ON_TIMESTAMP_SET: MOMENT: {}", moment); + log::info!("ON_TIMESTAMP_SET: DURATION: {}", SlotDuration::get()); + log::info!("ON_TIMESTAMP_SET: SLOT BEACON: {}", SlotBeacon::slot()); + let timestamp_slot = moment / SlotDuration::get(); + log::info!("ON_TIMESTAMP_SET: TIMESTAMP SLOT: {}", timestamp_slot); + assert!( SlotBeacon::slot() as u64 == timestamp_slot, "Timestamp slot must match SlotBeacon slot" diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index 40843d410..6baa726f4 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -33,7 +33,7 @@ pub use sp_runtime::BuildStorage; pub mod weights; use { - cumulus_pallet_parachain_system::{RelayChainStateProof, RelayNumberStrictlyIncreases}, + cumulus_pallet_parachain_system::{RelayChainStateProof, RelayNumberMonotonicallyIncreases, RelayNumberStrictlyIncreases}, cumulus_primitives_core::{ relay_chain::{self, SessionIndex}, AggregateMessageOrigin, BodyId, ParaId, @@ -498,7 +498,7 @@ impl cumulus_pallet_parachain_system::Config for Runtime { type ReservedDmpWeight = ReservedDmpWeight; type XcmpMessageHandler = XcmpQueue; type ReservedXcmpWeight = ReservedXcmpWeight; - type CheckAssociatedRelayNumber = RelayNumberStrictlyIncreases; + type CheckAssociatedRelayNumber = RelayNumberMonotonicallyIncreases; type ConsensusHook = ConsensusHook; } pub struct ParaSlotProvider; @@ -510,7 +510,7 @@ impl Get<(Slot, SlotDuration)> for ParaSlotProvider { } impl pallet_async_backing::Config for Runtime { - type AllowMultipleBlocksPerSlot = ConstBool; + type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; } From 147653b7a2322763dd13b00a35e2599607a82bbd Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 5 Mar 2024 13:22:36 -0800 Subject: [PATCH 03/35] fix zombie test --- test/suites/para/test_tanssi_containers.ts | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/suites/para/test_tanssi_containers.ts b/test/suites/para/test_tanssi_containers.ts index c2bad36d3..386b9b4a2 100644 --- a/test/suites/para/test_tanssi_containers.ts +++ b/test/suites/para/test_tanssi_containers.ts @@ -263,11 +263,11 @@ describeSuite({ expect(registered3.toJSON().includes(2002)).to.be.false; // The node starts one session before the container chain is in registered list - await waitSessions(context, paraApi, 1); + /* await waitSessions(context, paraApi, 1); // Not registered yet, still pending const registered4 = await paraApi.query.registrar.registeredParaIds(); // TODO: fix once we have types - expect(registered4.toJSON().includes(2002)).to.be.false; + expect(registered4.toJSON().includes(2002)).to.be.false; */ await waitSessions(context, paraApi, 1); // Check that registered para ids contains 2002 From 24633fa768314e542ec9dae9015f055b721f23ab Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 5 Mar 2024 13:23:08 -0800 Subject: [PATCH 04/35] set capacity to 1 in dancebox --- runtime/dancebox/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index 5490f6daa..2867f309c 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -479,7 +479,7 @@ parameter_types! { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 2; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 1; pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< From 97f604aab854a198db2959bcbbf18796b134e7c2 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Thu, 7 Mar 2024 12:38:12 -0800 Subject: [PATCH 05/35] fix dev nodes --- container-chains/templates/frontier/node/src/service.rs | 2 +- container-chains/templates/simple/node/src/service.rs | 2 +- node/src/service.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/container-chains/templates/frontier/node/src/service.rs b/container-chains/templates/frontier/node/src/service.rs index 1fe9af91b..ee04b3d92 100644 --- a/container-chains/templates/frontier/node/src/service.rs +++ b/container-chains/templates/frontier/node/src/service.rs @@ -412,7 +412,7 @@ pub async fn start_dev_node( timestamp.into(), SlotDuration::from_millis(slot_duration), ); - let relay_slot = u64::from(*relay_slot).saturating_mul(2); + let relay_slot = u64::from(*relay_slot); let downward_xcm_receiver = downward_xcm_receiver.clone(); let hrmp_xcm_receiver = hrmp_xcm_receiver.clone(); diff --git a/container-chains/templates/simple/node/src/service.rs b/container-chains/templates/simple/node/src/service.rs index ef7907164..61480f909 100644 --- a/container-chains/templates/simple/node/src/service.rs +++ b/container-chains/templates/simple/node/src/service.rs @@ -272,7 +272,7 @@ pub async fn start_dev_node( timestamp.into(), SlotDuration::from_millis(slot_duration), ); - let relay_slot = u64::from(*relay_slot).saturating_mul(2); + let relay_slot = u64::from(*relay_slot); let downward_xcm_receiver = downward_xcm_receiver.clone(); let hrmp_xcm_receiver = hrmp_xcm_receiver.clone(); diff --git a/node/src/service.rs b/node/src/service.rs index 3488c716a..ac65900d2 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -1083,7 +1083,7 @@ pub fn start_dev_node( timestamp.into(), slot_duration, ); - let relay_slot = u64::from(*relay_slot).saturating_mul(2); + let relay_slot = u64::from(*relay_slot); let downward_xcm_receiver = downward_xcm_receiver.clone(); let hrmp_xcm_receiver = hrmp_xcm_receiver.clone(); From c5f5e6cdf050d86d5f532ac14edcfd5c126919dc Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Thu, 7 Mar 2024 12:39:03 -0800 Subject: [PATCH 06/35] set six seconds in all runtimes --- container-chains/templates/frontier/runtime/src/lib.rs | 6 +++--- container-chains/templates/simple/runtime/src/lib.rs | 6 +++--- runtime/dancebox/src/lib.rs | 6 +++--- runtime/flashbox/src/lib.rs | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/container-chains/templates/frontier/runtime/src/lib.rs b/container-chains/templates/frontier/runtime/src/lib.rs index d64fe0658..15269ac8c 100644 --- a/container-chains/templates/frontier/runtime/src/lib.rs +++ b/container-chains/templates/frontier/runtime/src/lib.rs @@ -334,7 +334,7 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { /// up by `pallet_aura` to implement `fn slot_duration()`. /// /// Change this to adjust the block time. -pub const MILLISECS_PER_BLOCK: u64 = 12000; +pub const MILLISECS_PER_BLOCK: u64 = 6000; // NOTE: Currently it is not possible to change the slot duration after the chain has started. // Attempting to do so will brick block production. @@ -512,8 +512,8 @@ parameter_types! { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 1; -pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; +pub const BLOCK_PROCESSING_VELOCITY: u32 = 2; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< Runtime, diff --git a/container-chains/templates/simple/runtime/src/lib.rs b/container-chains/templates/simple/runtime/src/lib.rs index 782d95335..519d5c626 100644 --- a/container-chains/templates/simple/runtime/src/lib.rs +++ b/container-chains/templates/simple/runtime/src/lib.rs @@ -226,7 +226,7 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { /// up by `pallet_aura` to implement `fn slot_duration()`. /// /// Change this to adjust the block time. -pub const MILLISECS_PER_BLOCK: u64 = 12000; +pub const MILLISECS_PER_BLOCK: u64 = 6000; // NOTE: Currently it is not possible to change the slot duration after the chain has started. // Attempting to do so will brick block production. @@ -402,8 +402,8 @@ parameter_types! { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 1; -pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; +pub const BLOCK_PROCESSING_VELOCITY: u32 = 2; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< Runtime, diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index 2867f309c..4cbf6fb03 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -231,7 +231,7 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { /// up by `pallet_aura` to implement `fn slot_duration()`. /// /// Change this to adjust the block time. -pub const MILLISECS_PER_BLOCK: u64 = 12000; +pub const MILLISECS_PER_BLOCK: u64 = 6000; // NOTE: Currently it is not possible to change the slot duration after the chain has started. // Attempting to do so will brick block production. @@ -479,8 +479,8 @@ parameter_types! { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 1; -pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; +pub const BLOCK_PROCESSING_VELOCITY: u32 = 2; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< Runtime, diff --git a/runtime/flashbox/src/lib.rs b/runtime/flashbox/src/lib.rs index 3768fd07c..fb876501c 100644 --- a/runtime/flashbox/src/lib.rs +++ b/runtime/flashbox/src/lib.rs @@ -217,7 +217,7 @@ pub const VERSION: RuntimeVersion = RuntimeVersion { /// up by `pallet_aura` to implement `fn slot_duration()`. /// /// Change this to adjust the block time. -pub const MILLISECS_PER_BLOCK: u64 = 12000; +pub const MILLISECS_PER_BLOCK: u64 = 6000; // NOTE: Currently it is not possible to change the slot duration after the chain has started. // Attempting to do so will brick block production. From 41f882b346aff6284bac4e149f7b0f1b222d2d61 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Thu, 7 Mar 2024 12:39:25 -0800 Subject: [PATCH 07/35] add async backing params to zombienet config --- test/configs/zombieTanssi.json | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/test/configs/zombieTanssi.json b/test/configs/zombieTanssi.json index 1f680af76..34cbe6bad 100644 --- a/test/configs/zombieTanssi.json +++ b/test/configs/zombieTanssi.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", @@ -46,6 +61,7 @@ }, { "name": "Collator2000-01", + "ws_port": "9960", "command": "../target/release/tanssi-node", "args": ["--no-hardware-benchmarks", "--database=paritydb", "--wasmtime-precompiled=wasm"] }, @@ -56,6 +72,7 @@ }, { "name": "Collator2001-01", + "ws_port": "9961", "command": "../target/release/tanssi-node", "args": ["--no-hardware-benchmarks", "--database=paritydb", "--wasmtime-precompiled=wasm"] }, @@ -66,6 +83,7 @@ }, { "name": "Collator2002-01", + "ws_port": "9962", "command": "../target/release/tanssi-node", "args": ["--no-hardware-benchmarks", "--database=paritydb", "--wasmtime-precompiled=wasm"] }, From a03f77077e40d0fd7cb2e7168778533ee768459f Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 8 Mar 2024 09:19:40 -0800 Subject: [PATCH 08/35] fix some zombie tests --- test/suites/para/test_tanssi_containers.ts | 8 ++++---- test/suites/parathreads/test_tanssi_parathreads.ts | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/test/suites/para/test_tanssi_containers.ts b/test/suites/para/test_tanssi_containers.ts index 700cefef2..8fad045f4 100644 --- a/test/suites/para/test_tanssi_containers.ts +++ b/test/suites/para/test_tanssi_containers.ts @@ -310,7 +310,7 @@ describeSuite({ it({ id: "T15", title: "Deregister container chain 2002, collators should move to tanssi", - timeout: 300000, + timeout: 600000, test: async function () { const keyring = new Keyring({ type: "sr25519" }); const alice = keyring.addFromUri("//Alice", { name: "Alice default" }); @@ -358,12 +358,12 @@ describeSuite({ // TODO: this passes if only 2 authors are creating blocks, think a way to test that case await countUniqueBlockAuthors(paraApi, blockNumber, blockNumber2002Start - 1, 4); - expect(sessionPeriod * 5 < blockNumber2002End, "2002 should have deregistered after first rotation"); - expect(sessionPeriod * 10 > blockNumber2002End, "2002 should have deregistered before second rotation"); + expect(sessionPeriod * 10 < blockNumber2002End, "2002 should have deregistered after first rotation"); + expect(sessionPeriod * 20 > blockNumber2002End, "2002 should have deregistered before second rotation"); // While 2002 is live: 2 authors (the other 2 went to container chain 2002) // We take from the first block that rotates, otherwise rotation kicks in - await countUniqueBlockAuthors(paraApi, sessionPeriod * 5, blockNumber2002End - 1, 2); + await countUniqueBlockAuthors(paraApi, sessionPeriod * 10, blockNumber2002End - 1, 2); // Need to wait one session because the following blocks don't exist yet await waitSessions(context, paraApi, 1); diff --git a/test/suites/parathreads/test_tanssi_parathreads.ts b/test/suites/parathreads/test_tanssi_parathreads.ts index 5824145ab..415ab9119 100644 --- a/test/suites/parathreads/test_tanssi_parathreads.ts +++ b/test/suites/parathreads/test_tanssi_parathreads.ts @@ -258,7 +258,7 @@ describeSuite({ it({ id: "T12", title: "Check block frequency of parathreads", - timeout: 120000, + timeout: 240000, test: async function () { // Wait 1 session so that parathreads have produced at least a few blocks each await waitSessions(context, paraApi, 2); From b338b8c1116053121424f452af1a9594d6c2b846 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Sun, 10 Mar 2024 20:01:43 -0700 Subject: [PATCH 09/35] fmt and cleanup --- client/consensus/src/collators/lookahead.rs | 813 +++++++++--------- .../templates/frontier/runtime/src/lib.rs | 15 +- .../templates/simple/runtime/src/lib.rs | 15 +- node/src/service.rs | 10 +- runtime/dancebox/src/lib.rs | 16 +- 5 files changed, 437 insertions(+), 432 deletions(-) diff --git a/client/consensus/src/collators/lookahead.rs b/client/consensus/src/collators/lookahead.rs index dc7ca3a38..35d0f2e74 100644 --- a/client/consensus/src/collators/lookahead.rs +++ b/client/consensus/src/collators/lookahead.rs @@ -34,18 +34,16 @@ use async_backing_primitives::UnincludedSegmentApi; use cumulus_client_collator::service::ServiceInterface as CollatorServiceInterface; use cumulus_client_consensus_common::{ - self as consensus_common, load_abridged_host_configuration, ParachainBlockImportMarker, - ParentSearchParams, + self as consensus_common, load_abridged_host_configuration, ParachainBlockImportMarker, + ParentSearchParams, }; use cumulus_client_consensus_proposer::ProposerInterface; -use cumulus_primitives_core::{ - relay_chain::Hash as PHash, CollectCollationInfo, PersistedValidationData, -}; +use cumulus_primitives_core::{relay_chain::Hash as PHash, PersistedValidationData}; use cumulus_relay_chain_interface::RelayChainInterface; use parity_scale_codec::{Codec, Encode}; use polkadot_node_primitives::SubmitCollationParams; use polkadot_node_subsystem::messages::{ - CollationGenerationMessage, RuntimeApiMessage, RuntimeApiRequest, + CollationGenerationMessage, RuntimeApiMessage, RuntimeApiRequest, }; use polkadot_overseer::Handle as OverseerHandle; use polkadot_primitives::{CollatorPair, Id as ParaId, OccupiedCoreAssumption}; @@ -67,63 +65,63 @@ use sp_runtime::traits::{Block as BlockT, Header as HeaderT, Member}; use std::{convert::TryFrom, error::Error, sync::Arc, time::Duration}; use crate::{ - collators::{self as collator_util, SlotClaim, tanssi_claim_slot}, consensus_orchestrator::RetrieveAuthoritiesFromOrchestrator, + collators::{self as collator_util, tanssi_claim_slot, SlotClaim}, + consensus_orchestrator::RetrieveAuthoritiesFromOrchestrator, OrchestratorAuraWorkerAuxData, }; /// Parameters for [`run`]. pub struct Params { - pub create_inherent_data_providers: CIDP, + pub create_inherent_data_providers: CIDP, pub get_orchestrator_aux_data: GOH, - pub block_import: BI, - pub para_client: Arc, - pub para_backend: Arc, - pub relay_client: RClient, - pub code_hash_provider: CHP, - pub sync_oracle: SO, - pub keystore: KeystorePtr, - pub collator_key: CollatorPair, - pub para_id: ParaId, - pub overseer_handle: OverseerHandle, - pub slot_duration: SlotDuration, - pub relay_chain_slot_duration: Duration, - pub proposer: Proposer, - pub collator_service: CS, - pub authoring_duration: Duration, - pub force_authoring: bool, + pub block_import: BI, + pub para_client: Arc, + pub para_backend: Arc, + pub relay_client: RClient, + pub code_hash_provider: CHP, + pub sync_oracle: SO, + pub keystore: KeystorePtr, + pub collator_key: CollatorPair, + pub para_id: ParaId, + pub overseer_handle: OverseerHandle, + pub slot_duration: SlotDuration, + pub relay_chain_slot_duration: Duration, + pub proposer: Proposer, + pub collator_service: CS, + pub authoring_duration: Duration, + pub force_authoring: bool, } /// Run async-backing-friendly for Tanssi Aura. pub fn run( - mut params: Params, + mut params: Params, ) -> impl Future + Send + 'static where - Block: BlockT, - Client: ProvideRuntimeApi - + BlockOf - + AuxStore - + HeaderBackend - + BlockBackend - + Send - + Sync - + 'static, - Client::Api: UnincludedSegmentApi, - //AuraApi + CollectCollationInfo, //+ AuraUnincludedSegmentApi, - Backend: sc_client_api::Backend + 'static, - RClient: RelayChainInterface + Clone + 'static, + Block: BlockT, + Client: ProvideRuntimeApi + + BlockOf + + AuxStore + + HeaderBackend + + BlockBackend + + Send + + Sync + + 'static, + Client::Api: UnincludedSegmentApi, + Backend: sc_client_api::Backend + 'static, + RClient: RelayChainInterface + Clone + 'static, CIDP: CreateInherentDataProviders + Send + 'static + Clone, CIDP::InherentDataProviders: Send + InherentDataProviderExt, - BI: BlockImport + ParachainBlockImportMarker + Send + Sync + 'static, - SO: SyncOracle + Send + Sync + Clone + 'static, - Proposer: ProposerInterface + Send + Sync + 'static, - CS: CollatorServiceInterface + Send + Sync + 'static, - CHP: consensus_common::ValidationCodeHashProvider + Send + 'static, - P: Pair + Sync + Send + 'static, - P::Public: AppPublic + Member + Codec, - P::Signature: TryFrom> + Member + Codec, + BI: BlockImport + ParachainBlockImportMarker + Send + Sync + 'static, + SO: SyncOracle + Send + Sync + Clone + 'static, + Proposer: ProposerInterface + Send + Sync + 'static, + CS: CollatorServiceInterface + Send + Sync + 'static, + CHP: consensus_common::ValidationCodeHashProvider + Send + 'static, + P: Pair + Sync + Send + 'static, + P::Public: AppPublic + Member + Codec, + P::Signature: TryFrom> + Member + Codec, GOH: RetrieveAuthoritiesFromOrchestrator< Block, (PHash, PersistedValidationData), @@ -133,383 +131,390 @@ where + Sync + Send, { - // This is an arbitrary value which is likely guaranteed to exceed any reasonable - // limit, as it would correspond to 10 non-included blocks. - // - // Since we only search for parent blocks which have already been imported, - // we can guarantee that all imported blocks respect the unincluded segment - // rules specified by the parachain's runtime and thus will never be too deep. - const PARENT_SEARCH_DEPTH: usize = 10; - - log::info!("LOOKAHEAD COLLATOR RUNNING..."); - - async move { - cumulus_client_collator::initialize_collator_subsystems( - &mut params.overseer_handle, - params.collator_key, - params.para_id, - ) - .await; - - let mut import_notifications = match params.relay_client.import_notification_stream().await - { - Ok(s) => s, - Err(err) => { - tracing::error!( - target: crate::LOG_TARGET, - ?err, - "Failed to initialize consensus: no relay chain import notification stream" - ); - - return - }, - }; - - let mut collator = { - let params = collator_util::Params { - create_inherent_data_providers: params.create_inherent_data_providers.clone(), - block_import: params.block_import, - relay_client: params.relay_client.clone(), - keystore: params.keystore.clone(), - para_id: params.para_id, - proposer: params.proposer, - collator_service: params.collator_service, - }; - - collator_util::Collator::::new(params) - }; - - while let Some(relay_parent_header) = import_notifications.next().await { - let relay_parent = relay_parent_header.hash(); - - if !is_para_scheduled(relay_parent, params.para_id, &mut params.overseer_handle).await { - tracing::trace!( - target: crate::LOG_TARGET, - ?relay_parent, - ?params.para_id, - "Para is not scheduled on any core, skipping import notification", - ); - - continue - } - - let max_pov_size = match params - .relay_client - .persisted_validation_data( - relay_parent, - params.para_id, - OccupiedCoreAssumption::Included, - ) - .await - { - Ok(None) => continue, - Ok(Some(pvd)) => pvd.max_pov_size, - Err(err) => { - tracing::error!(target: crate::LOG_TARGET, ?err, "Failed to gather information from relay-client"); - continue - }, - }; - - let parent_search_params = ParentSearchParams { - relay_parent, - para_id: params.para_id, - ancestry_lookback: max_ancestry_lookback(relay_parent, ¶ms.relay_client).await, - max_depth: PARENT_SEARCH_DEPTH, - ignore_alternative_branches: true, - }; - - let potential_parents = - cumulus_client_consensus_common::find_potential_parents::( - parent_search_params, - &*params.para_backend, - ¶ms.relay_client, - ) - .await; - - let mut potential_parents = match potential_parents { - Err(e) => { - tracing::error!( - target: crate::LOG_TARGET, - ?relay_parent, - err = ?e, - "Could not fetch potential parents to build upon" - ); - - continue - }, - Ok(x) => x, - }; - - let included_block = match potential_parents.iter().find(|x| x.depth == 0) { - None => continue, // also serves as an `is_empty` check. - Some(b) => b.hash, - }; - - let para_client = &*params.para_client; - let keystore = ¶ms.keystore; - let can_build_upon = |slot_now, block_hash, aux_data| { - can_build_upon::<_, _, P>( - slot_now, - aux_data, - block_hash, - included_block, - params.force_authoring, - para_client, - &keystore, - ) - }; - - // Sort by depth, ascending, to choose the longest chain. - // - // If the longest chain has space, build upon that. Otherwise, don't - // build at all. - potential_parents.sort_by_key(|a| a.depth); - let initial_parent = match potential_parents.pop() { - None => continue, - Some(p) => p, - }; - - // Build in a loop until not allowed. Note that the authorities can change - // at any block, so we need to re-claim our slot every time. - let mut parent_hash = initial_parent.hash; - let mut parent_header = initial_parent.header; - let overseer_handle = &mut params.overseer_handle; - - // This needs to change to support elastic scaling, but for continuously - // scheduled chains this ensures that the backlog will grow steadily. - for n_built in 0..2 { - let validation_data = PersistedValidationData { - parent_head: parent_header.encode().into(), - relay_parent_number: *relay_parent_header.number(), - relay_parent_storage_root: *relay_parent_header.state_root(), - max_pov_size, - }; - - // Retrieve authorities that are able to produce the block - let aux_data = match params - .get_orchestrator_aux_data - .retrieve_authorities_from_orchestrator( - parent_hash, - (relay_parent_header.hash(), validation_data.clone()), - ) - .await - { - Err(e) => { - tracing::error!(target: crate::LOG_TARGET, ?e); - break; - }, - Ok(h) => h, - }; - - let inherent_providers = match params - .create_inherent_data_providers - .create_inherent_data_providers( - parent_hash, - (relay_parent_header.hash(), validation_data.clone()), + // This is an arbitrary value which is likely guaranteed to exceed any reasonable + // limit, as it would correspond to 10 non-included blocks. + // + // Since we only search for parent blocks which have already been imported, + // we can guarantee that all imported blocks respect the unincluded segment + // rules specified by the parachain's runtime and thus will never be too deep. + const PARENT_SEARCH_DEPTH: usize = 10; + + log::info!("LOOKAHEAD COLLATOR RUNNING..."); + + async move { + cumulus_client_collator::initialize_collator_subsystems( + &mut params.overseer_handle, + params.collator_key, + params.para_id, + ) + .await; + + let mut import_notifications = match params.relay_client.import_notification_stream().await + { + Ok(s) => s, + Err(err) => { + tracing::error!( + target: crate::LOG_TARGET, + ?err, + "Failed to initialize consensus: no relay chain import notification stream" + ); + + return; + } + }; + + let mut collator = { + let params = collator_util::Params { + create_inherent_data_providers: params.create_inherent_data_providers.clone(), + block_import: params.block_import, + relay_client: params.relay_client.clone(), + keystore: params.keystore.clone(), + para_id: params.para_id, + proposer: params.proposer, + collator_service: params.collator_service, + }; + + collator_util::Collator::::new(params) + }; + + while let Some(relay_parent_header) = import_notifications.next().await { + let relay_parent = relay_parent_header.hash(); + + if !is_para_scheduled(relay_parent, params.para_id, &mut params.overseer_handle).await { + tracing::trace!( + target: crate::LOG_TARGET, + ?relay_parent, + ?params.para_id, + "Para is not scheduled on any core, skipping import notification", + ); + + continue; + } + + let max_pov_size = match params + .relay_client + .persisted_validation_data( + relay_parent, + params.para_id, + OccupiedCoreAssumption::Included, + ) + .await + { + Ok(None) => continue, + Ok(Some(pvd)) => pvd.max_pov_size, + Err(err) => { + tracing::error!(target: crate::LOG_TARGET, ?err, "Failed to gather information from relay-client"); + continue; + } + }; + + let parent_search_params = ParentSearchParams { + relay_parent, + para_id: params.para_id, + ancestry_lookback: max_ancestry_lookback(relay_parent, ¶ms.relay_client).await, + max_depth: PARENT_SEARCH_DEPTH, + ignore_alternative_branches: true, + }; + + let potential_parents = + cumulus_client_consensus_common::find_potential_parents::( + parent_search_params, + &*params.para_backend, + ¶ms.relay_client, ) - .await - { - Err(e) => { - tracing::error!(target: crate::LOG_TARGET, ?e); - break; - }, - Ok(h) => h, - }; - - let mut slot_claim = match can_build_upon(inherent_providers.slot(), parent_header.clone(), aux_data).await { - Ok(None) => break, - Err(e) => { - tracing::error!(target: crate::LOG_TARGET, ?e); - break; - }, - Ok(Some(c)) => c, - }; - - tracing::debug!( - target: crate::LOG_TARGET, - ?relay_parent, - unincluded_segment_len = initial_parent.depth + n_built, - "Slot claimed. Building" - ); - - // Build and announce collations recursively until - // `can_build_upon` fails or building a collation fails. - let (parachain_inherent_data, other_inherent_data) = match collator - .create_inherent_data( - relay_parent, - &validation_data, - parent_hash, - None, - ) - .await - { - Err(err) => { - tracing::error!(target: crate::LOG_TARGET, ?err); - break - }, - Ok(x) => x, - }; - - let validation_code_hash = match params.code_hash_provider.code_hash_at(parent_hash) - { - None => { - tracing::error!(target: crate::LOG_TARGET, ?parent_hash, "Could not fetch validation code hash"); - break - }, - Some(v) => v, - }; - - match collator - .collate( - &parent_header, - &mut slot_claim, - None, - (parachain_inherent_data, other_inherent_data), - params.authoring_duration, - // Set the block limit to 50% of the maximum PoV size. - // - // TODO: If we got benchmarking that includes the proof size, - // we should be able to use the maximum pov size. - (validation_data.max_pov_size / 2) as usize, - ) - .await - { - Ok(Some((collation, block_data, new_block_hash))) => { - // Here we are assuming that the import logic protects against equivocations - // and provides sybil-resistance, as it should. - collator.collator_service().announce_block(new_block_hash, None); - - // Send a submit-collation message to the collation generation subsystem, - // which then distributes this to validators. - // - // Here we are assuming that the leaf is imported, as we've gotten an - // import notification. - overseer_handle - .send_msg( - CollationGenerationMessage::SubmitCollation( - SubmitCollationParams { - relay_parent, - collation, - parent_head: parent_header.encode().into(), - validation_code_hash, - result_sender: None, - }, - ), - "SubmitCollation", - ) - .await; - - parent_hash = new_block_hash; - parent_header = block_data.into_header(); - }, - Ok(None) => { - tracing::debug!(target: crate::LOG_TARGET, "Lookahead collator: No block proposal"); - } - Err(err) => { - tracing::error!(target: crate::LOG_TARGET, ?err); - break - }, - } - } - } - } + .await; + + let mut potential_parents = match potential_parents { + Err(e) => { + tracing::error!( + target: crate::LOG_TARGET, + ?relay_parent, + err = ?e, + "Could not fetch potential parents to build upon" + ); + + continue; + } + Ok(x) => x, + }; + + let included_block = match potential_parents.iter().find(|x| x.depth == 0) { + None => continue, // also serves as an `is_empty` check. + Some(b) => b.hash, + }; + + let para_client = &*params.para_client; + let keystore = ¶ms.keystore; + let can_build_upon = |slot_now, block_hash, aux_data| { + can_build_upon::<_, _, P>( + slot_now, + aux_data, + block_hash, + included_block, + params.force_authoring, + para_client, + &keystore, + ) + }; + + // Sort by depth, ascending, to choose the longest chain. + // + // If the longest chain has space, build upon that. Otherwise, don't + // build at all. + potential_parents.sort_by_key(|a| a.depth); + let initial_parent = match potential_parents.pop() { + None => continue, + Some(p) => p, + }; + + // Build in a loop until not allowed. Note that the authorities can change + // at any block, so we need to re-claim our slot every time. + let mut parent_hash = initial_parent.hash; + let mut parent_header = initial_parent.header; + let overseer_handle = &mut params.overseer_handle; + + // This needs to change to support elastic scaling, but for continuously + // scheduled chains this ensures that the backlog will grow steadily. + for n_built in 0..2 { + let validation_data = PersistedValidationData { + parent_head: parent_header.encode().into(), + relay_parent_number: *relay_parent_header.number(), + relay_parent_storage_root: *relay_parent_header.state_root(), + max_pov_size, + }; + + // Retrieve authorities that are able to produce the block + let aux_data = match params + .get_orchestrator_aux_data + .retrieve_authorities_from_orchestrator( + parent_hash, + (relay_parent_header.hash(), validation_data.clone()), + ) + .await + { + Err(e) => { + tracing::error!(target: crate::LOG_TARGET, ?e); + break; + } + Ok(h) => h, + }; + + let inherent_providers = match params + .create_inherent_data_providers + .create_inherent_data_providers( + parent_hash, + (relay_parent_header.hash(), validation_data.clone()), + ) + .await + { + Err(e) => { + tracing::error!(target: crate::LOG_TARGET, ?e); + break; + } + Ok(h) => h, + }; + + let mut slot_claim = match can_build_upon( + inherent_providers.slot(), + parent_header.clone(), + aux_data, + ) + .await + { + Ok(None) => break, + Err(e) => { + tracing::error!(target: crate::LOG_TARGET, ?e); + break; + } + Ok(Some(c)) => c, + }; + + tracing::debug!( + target: crate::LOG_TARGET, + ?relay_parent, + unincluded_segment_len = initial_parent.depth + n_built, + "Slot claimed. Building" + ); + + // Build and announce collations recursively until + // `can_build_upon` fails or building a collation fails. + let (parachain_inherent_data, other_inherent_data) = match collator + .create_inherent_data(relay_parent, &validation_data, parent_hash, None) + .await + { + Err(err) => { + tracing::error!(target: crate::LOG_TARGET, ?err); + break; + } + Ok(x) => x, + }; + + let validation_code_hash = match params.code_hash_provider.code_hash_at(parent_hash) + { + None => { + tracing::error!(target: crate::LOG_TARGET, ?parent_hash, "Could not fetch validation code hash"); + break; + } + Some(v) => v, + }; + + match collator + .collate( + &parent_header, + &mut slot_claim, + None, + (parachain_inherent_data, other_inherent_data), + params.authoring_duration, + // Set the block limit to 50% of the maximum PoV size. + // + // TODO: If we got benchmarking that includes the proof size, + // we should be able to use the maximum pov size. + (validation_data.max_pov_size / 2) as usize, + ) + .await + { + Ok(Some((collation, block_data, new_block_hash))) => { + // Here we are assuming that the import logic protects against equivocations + // and provides sybil-resistance, as it should. + collator + .collator_service() + .announce_block(new_block_hash, None); + + // Send a submit-collation message to the collation generation subsystem, + // which then distributes this to validators. + // + // Here we are assuming that the leaf is imported, as we've gotten an + // import notification. + overseer_handle + .send_msg( + CollationGenerationMessage::SubmitCollation( + SubmitCollationParams { + relay_parent, + collation, + parent_head: parent_header.encode().into(), + validation_code_hash, + result_sender: None, + }, + ), + "SubmitCollation", + ) + .await; + + parent_hash = new_block_hash; + parent_header = block_data.into_header(); + } + Ok(None) => { + tracing::debug!(target: crate::LOG_TARGET, "Lookahead collator: No block proposal"); + } + Err(err) => { + tracing::error!(target: crate::LOG_TARGET, ?err); + break; + } + } + } + } + } } -// Checks if we own the slot at the given block and whether there +// Checks if we own the slot at the given block and whether there // is space in the unincluded segment. async fn can_build_upon( - slot: Slot, - aux_data: OrchestratorAuraWorkerAuxData

, - parent_header: Block::Header, - included_block: Block::Hash, - force_authoring: bool, - client: &Client, - keystore: &KeystorePtr, + slot: Slot, + aux_data: OrchestratorAuraWorkerAuxData

, + parent_header: Block::Header, + included_block: Block::Hash, + force_authoring: bool, + client: &Client, + keystore: &KeystorePtr, ) -> Result>, Box> where - Client: ProvideRuntimeApi, - Client::Api: UnincludedSegmentApi,//AuraApi, //+ AuraUnincludedSegmentApi, - P: Pair + Send + Sync + 'static, - P::Public: Codec + std::fmt::Debug, - P::Signature: Codec, + Client: ProvideRuntimeApi, + Client::Api: UnincludedSegmentApi, + P: Pair + Send + Sync + 'static, + P::Public: Codec + std::fmt::Debug, + P::Signature: Codec, { - let runtime_api = client.runtime_api(); - let slot_claim = tanssi_claim_slot::(aux_data, &parent_header, slot, force_authoring, keystore); - - // Here we lean on the property that building on an empty unincluded segment must always - // be legal. Skipping the runtime API query here allows us to seamlessly run this - // collator against chains which have not yet upgraded their runtime. - if parent_header.hash() != included_block { - if !runtime_api.can_build_upon(parent_header.hash(), included_block, slot)? { - return Ok(None) - } - } - - slot_claim + let runtime_api = client.runtime_api(); + let slot_claim = + tanssi_claim_slot::(aux_data, &parent_header, slot, force_authoring, keystore); + + // Here we lean on the property that building on an empty unincluded segment must always + // be legal. Skipping the runtime API query here allows us to seamlessly run this + // collator against chains which have not yet upgraded their runtime. + if parent_header.hash() != included_block { + if !runtime_api.can_build_upon(parent_header.hash(), included_block, slot)? { + return Ok(None); + } + } + + slot_claim } /// Reads allowed ancestry length parameter from the relay chain storage at the given relay parent. /// /// Falls back to 0 in case of an error. async fn max_ancestry_lookback( - relay_parent: PHash, - relay_client: &impl RelayChainInterface, + relay_parent: PHash, + relay_client: &impl RelayChainInterface, ) -> usize { - match load_abridged_host_configuration(relay_parent, relay_client).await { - Ok(Some(config)) => config.async_backing_params.allowed_ancestry_len as usize, - Ok(None) => { - tracing::error!( - target: crate::LOG_TARGET, - "Active config is missing in relay chain storage", - ); - 0 - }, - Err(err) => { - tracing::error!( - target: crate::LOG_TARGET, - ?err, - ?relay_parent, - "Failed to read active config from relay chain client", - ); - 0 - }, - } + match load_abridged_host_configuration(relay_parent, relay_client).await { + Ok(Some(config)) => config.async_backing_params.allowed_ancestry_len as usize, + Ok(None) => { + tracing::error!( + target: crate::LOG_TARGET, + "Active config is missing in relay chain storage", + ); + 0 + } + Err(err) => { + tracing::error!( + target: crate::LOG_TARGET, + ?err, + ?relay_parent, + "Failed to read active config from relay chain client", + ); + 0 + } + } } // Checks if there exists a scheduled core for the para at the provided relay parent. // // Falls back to `false` in case of an error. async fn is_para_scheduled( - relay_parent: PHash, - para_id: ParaId, - overseer_handle: &mut OverseerHandle, + relay_parent: PHash, + para_id: ParaId, + overseer_handle: &mut OverseerHandle, ) -> bool { - let (tx, rx) = oneshot::channel(); - let request = RuntimeApiRequest::AvailabilityCores(tx); - overseer_handle - .send_msg(RuntimeApiMessage::Request(relay_parent, request), "LookaheadCollator") - .await; - - let cores = match rx.await { - Ok(Ok(cores)) => cores, - Ok(Err(error)) => { - tracing::error!( - target: crate::LOG_TARGET, - ?error, - ?relay_parent, - "Failed to query availability cores runtime API", - ); - return false - }, - Err(oneshot::Canceled) => { - tracing::error!( - target: crate::LOG_TARGET, - ?relay_parent, - "Sender for availability cores runtime request dropped", - ); - return false - }, - }; - - cores.iter().any(|core| core.para_id() == Some(para_id)) -} \ No newline at end of file + let (tx, rx) = oneshot::channel(); + let request = RuntimeApiRequest::AvailabilityCores(tx); + overseer_handle + .send_msg( + RuntimeApiMessage::Request(relay_parent, request), + "LookaheadCollator", + ) + .await; + + let cores = match rx.await { + Ok(Ok(cores)) => cores, + Ok(Err(error)) => { + tracing::error!( + target: crate::LOG_TARGET, + ?error, + ?relay_parent, + "Failed to query availability cores runtime API", + ); + return false; + } + Err(oneshot::Canceled) => { + tracing::error!( + target: crate::LOG_TARGET, + ?relay_parent, + "Sender for availability cores runtime request dropped", + ); + return false; + } + }; + + cores.iter().any(|core| core.para_id() == Some(para_id)) +} diff --git a/container-chains/templates/frontier/runtime/src/lib.rs b/container-chains/templates/frontier/runtime/src/lib.rs index 15269ac8c..b1740ed15 100644 --- a/container-chains/templates/frontier/runtime/src/lib.rs +++ b/container-chains/templates/frontier/runtime/src/lib.rs @@ -35,7 +35,6 @@ pub mod xcm_config; use { crate::precompiles::TemplatePrecompiles, - cumulus_pallet_parachain_system::RelayNumberStrictlyIncreases, cumulus_primitives_core::AggregateMessageOrigin, fp_account::EthereumSignature, fp_evm::weight_per_gas, @@ -1073,13 +1072,13 @@ impl_runtime_apis! { } impl async_backing_primitives::UnincludedSegmentApi for Runtime { - fn can_build_upon( - included_hash: ::Hash, - slot: async_backing_primitives::Slot, - ) -> bool { - ConsensusHook::can_build_upon(included_hash, slot) - } - } + fn can_build_upon( + included_hash: ::Hash, + slot: async_backing_primitives::Slot, + ) -> bool { + ConsensusHook::can_build_upon(included_hash, slot) + } + } impl sp_genesis_builder::GenesisBuilder for Runtime { fn create_default_config() -> Vec { diff --git a/container-chains/templates/simple/runtime/src/lib.rs b/container-chains/templates/simple/runtime/src/lib.rs index 519d5c626..42dcddd3f 100644 --- a/container-chains/templates/simple/runtime/src/lib.rs +++ b/container-chains/templates/simple/runtime/src/lib.rs @@ -33,7 +33,6 @@ pub mod migrations; pub use sp_runtime::{MultiAddress, Perbill, Permill}; use { - cumulus_pallet_parachain_system::RelayNumberStrictlyIncreases, cumulus_primitives_core::AggregateMessageOrigin, frame_support::{ construct_runtime, @@ -795,13 +794,13 @@ impl_runtime_apis! { } impl async_backing_primitives::UnincludedSegmentApi for Runtime { - fn can_build_upon( - included_hash: ::Hash, - slot: async_backing_primitives::Slot, - ) -> bool { - ConsensusHook::can_build_upon(included_hash, slot) - } - } + fn can_build_upon( + included_hash: ::Hash, + slot: async_backing_primitives::Slot, + ) -> bool { + ConsensusHook::can_build_upon(included_hash, slot) + } + } impl sp_genesis_builder::GenesisBuilder for Runtime { fn create_default_config() -> Vec { diff --git a/node/src/service.rs b/node/src/service.rs index ac65900d2..0707a8610 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -73,8 +73,10 @@ use { std::{sync::Arc, time::Duration}, substrate_prometheus_endpoint::Registry, tc_consensus::{ - collators::lookahead::{self as lookahead_tanssi_aura, Params as LookaheadTanssiAuraParams}, - collators::basic::{self as basic_tanssi_aura, Params as BasicTanssiAuraParams}, + collators::lookahead::{ + self as lookahead_tanssi_aura, Params as LookaheadTanssiAuraParams, + }, + //collators::basic::{self as basic_tanssi_aura, Params as BasicTanssiAuraParams}, OrchestratorAuraWorkerAuxData, }, tokio::sync::mpsc::{unbounded_channel, UnboundedSender}, @@ -833,7 +835,7 @@ fn start_consensus_container( //collation_request_receiver: None, }; - let fut = lookahead_tanssi_aura::run::(params); + let fut = lookahead_tanssi_aura::run::(params); spawner.spawn("tanssi-aura-container", None, fut); } @@ -964,7 +966,7 @@ fn start_consensus_orchestrator( }; //let fut = basic_tanssi_aura::run::(params); - let fut = lookahead_tanssi_aura::run::(params); + let fut = lookahead_tanssi_aura::run::(params); spawner.spawn("tanssi-aura", None, fut); } diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index 271231ddb..eea0fa298 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -33,7 +33,7 @@ pub use sp_runtime::BuildStorage; pub mod weights; use { - cumulus_pallet_parachain_system::{RelayChainStateProof, RelayNumberMonotonicallyIncreases, RelayNumberStrictlyIncreases}, + cumulus_pallet_parachain_system::{RelayChainStateProof, RelayNumberMonotonicallyIncreases}, cumulus_primitives_core::{ relay_chain::{self, SessionIndex}, AggregateMessageOrigin, BodyId, ParaId, @@ -1795,13 +1795,13 @@ impl_runtime_apis! { } impl async_backing_primitives::UnincludedSegmentApi for Runtime { - fn can_build_upon( - included_hash: ::Hash, - slot: async_backing_primitives::Slot, - ) -> bool { - ConsensusHook::can_build_upon(included_hash, slot) - } - } + fn can_build_upon( + included_hash: ::Hash, + slot: async_backing_primitives::Slot, + ) -> bool { + ConsensusHook::can_build_upon(included_hash, slot) + } + } impl sp_genesis_builder::GenesisBuilder for Runtime { fn create_default_config() -> Vec { From 5a81355247c3a07628399099aef9955d377e2c74 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Sun, 10 Mar 2024 20:15:41 -0700 Subject: [PATCH 10/35] toml fmt --- primitives/consensus/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/primitives/consensus/Cargo.toml b/primitives/consensus/Cargo.toml index 8d9b23032..7ca94d140 100644 --- a/primitives/consensus/Cargo.toml +++ b/primitives/consensus/Cargo.toml @@ -12,10 +12,10 @@ targets = [ "x86_64-unknown-linux-gnu" ] workspace = true [dependencies] -log = { workspace = true } cumulus-primitives-core = { workspace = true } frame-support = { workspace = true } frame-system = { workspace = true } +log = { workspace = true } parity-scale-codec = { workspace = true } scale-info = { workspace = true } From 98fe64fab3ccf81d43ef81b460add72b46640f1a Mon Sep 17 00:00:00 2001 From: girazoki Date: Mon, 11 Mar 2024 15:57:33 +0100 Subject: [PATCH 11/35] fix ts tests --- .../test_service_payment_removes_tank_money_and_burns.ts | 2 +- .../test_service_payment_removes_tank_money_and_refunds.ts | 2 +- .../test_services_payment_block_credit_buying_free_combined.ts | 2 +- .../services-payment/test_services_payment_block_credits.ts | 2 +- .../services-payment/test_services_payment_no_free_credits.ts | 2 +- test/suites/dev-tanssi/staking/test_staking_join.ts | 2 +- test/suites/dev-tanssi/staking/test_staking_session.ts | 2 +- test/suites/dev-tanssi/staking/test_staking_swap.ts | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_burns.ts b/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_burns.ts index e04d69006..3a609bbad 100644 --- a/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_burns.ts +++ b/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_burns.ts @@ -12,7 +12,7 @@ describeSuite({ testCases: ({ it, context }) => { let polkadotJs: ApiPromise; let alice: KeyringPair; - const blocksPerSession = 5n; + const blocksPerSession = 10n; const paraId2001 = 2001n; const costPerBlock = 1_000_000n; let balanceTankBefore; diff --git a/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_refunds.ts b/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_refunds.ts index dc6748686..e22b27ae2 100644 --- a/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_refunds.ts +++ b/test/suites/common-tanssi/services-payment/test_service_payment_removes_tank_money_and_refunds.ts @@ -12,7 +12,7 @@ describeSuite({ testCases: ({ it, context }) => { let polkadotJs: ApiPromise; let alice: KeyringPair; - const blocksPerSession = 5n; + const blocksPerSession = 10n; const paraId2001 = 2001n; const costPerBlock = 1_000_000n; let refundAddress; diff --git a/test/suites/common-tanssi/services-payment/test_services_payment_block_credit_buying_free_combined.ts b/test/suites/common-tanssi/services-payment/test_services_payment_block_credit_buying_free_combined.ts index 00695ed49..e927de756 100644 --- a/test/suites/common-tanssi/services-payment/test_services_payment_block_credit_buying_free_combined.ts +++ b/test/suites/common-tanssi/services-payment/test_services_payment_block_credit_buying_free_combined.ts @@ -11,7 +11,7 @@ describeSuite({ testCases: ({ it, context }) => { let polkadotJs: ApiPromise; let alice: KeyringPair; - const blocksPerSession = 5n; + const blocksPerSession = 10n; const paraId2000 = 2000n; const paraId2001 = 2001n; const costPerBlock = 1_000_000n; diff --git a/test/suites/common-tanssi/services-payment/test_services_payment_block_credits.ts b/test/suites/common-tanssi/services-payment/test_services_payment_block_credits.ts index 39aac86aa..a7172e40b 100644 --- a/test/suites/common-tanssi/services-payment/test_services_payment_block_credits.ts +++ b/test/suites/common-tanssi/services-payment/test_services_payment_block_credits.ts @@ -12,7 +12,7 @@ describeSuite({ testCases: ({ it, context }) => { let polkadotJs: ApiPromise; let alice: KeyringPair; - const blocksPerSession = 5n; + const blocksPerSession = 10n; beforeAll(async () => { polkadotJs = context.polkadotJs(); diff --git a/test/suites/common-tanssi/services-payment/test_services_payment_no_free_credits.ts b/test/suites/common-tanssi/services-payment/test_services_payment_no_free_credits.ts index 711d61316..700b3c945 100644 --- a/test/suites/common-tanssi/services-payment/test_services_payment_no_free_credits.ts +++ b/test/suites/common-tanssi/services-payment/test_services_payment_no_free_credits.ts @@ -15,7 +15,7 @@ describeSuite({ const paraId2001 = 2001n; const costPerSession = 100_000_000n; const costPerBlock = 1_000_000n; - const blocksPerSession = 5n; + const blocksPerSession = 10n; beforeAll(async () => { polkadotJs = context.polkadotJs(); alice = context.keyring.alice; diff --git a/test/suites/dev-tanssi/staking/test_staking_join.ts b/test/suites/dev-tanssi/staking/test_staking_join.ts index fd343ba22..3d919a357 100644 --- a/test/suites/dev-tanssi/staking/test_staking_join.ts +++ b/test/suites/dev-tanssi/staking/test_staking_join.ts @@ -14,7 +14,7 @@ describeSuite({ let alice: KeyringPair; let bob: KeyringPair; // TODO: don't hardcode the period here - const sessionPeriod = 5; + const sessionPeriod = 10; beforeAll(async () => { alice = context.keyring.alice; diff --git a/test/suites/dev-tanssi/staking/test_staking_session.ts b/test/suites/dev-tanssi/staking/test_staking_session.ts index 8ba6fb5f2..b61736d05 100644 --- a/test/suites/dev-tanssi/staking/test_staking_session.ts +++ b/test/suites/dev-tanssi/staking/test_staking_session.ts @@ -13,7 +13,7 @@ describeSuite({ let polkadotJs: ApiPromise; let alice: KeyringPair; // TODO: don't hardcode the period here - const sessionPeriod = 5; + const sessionPeriod = 10; beforeAll(async () => { alice = context.keyring.alice; diff --git a/test/suites/dev-tanssi/staking/test_staking_swap.ts b/test/suites/dev-tanssi/staking/test_staking_swap.ts index ae52124b9..c0303dee1 100644 --- a/test/suites/dev-tanssi/staking/test_staking_swap.ts +++ b/test/suites/dev-tanssi/staking/test_staking_swap.ts @@ -14,7 +14,7 @@ describeSuite({ let alice: KeyringPair; let bob: KeyringPair; // TODO: don't hardcode the period here - const sessionPeriod = 5; + const sessionPeriod = 10; beforeAll(async () => { alice = context.keyring.alice; From 77cb835544b5938d6460eabb6cd6b7e6b6346cac Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Mon, 11 Mar 2024 12:51:03 -0700 Subject: [PATCH 12/35] use custom branches --- Cargo.lock | 690 ++++++++++++++++++++++++++--------------------------- Cargo.toml | 348 +++++++++++++-------------- 2 files changed, 519 insertions(+), 519 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 208f36c41..a3ef753c4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -433,7 +433,7 @@ checksum = "9b34d609dfbaf33d6889b2b7106d3ca345eacad44200913df5ba02bfd31d2ba9" [[package]] name = "asset-test-utils" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "assets-common", "cumulus-pallet-parachain-system", @@ -468,7 +468,7 @@ dependencies = [ [[package]] name = "assets-common" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -492,7 +492,7 @@ dependencies = [ [[package]] name = "async-backing-primitives" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "sp-api", "sp-consensus-slots", @@ -788,7 +788,7 @@ dependencies = [ [[package]] name = "binary-merkle-tree" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "hash-db", "log", @@ -1013,7 +1013,7 @@ dependencies = [ [[package]] name = "bp-header-chain" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-runtime", "finality-grandpa", @@ -1030,7 +1030,7 @@ dependencies = [ [[package]] name = "bp-messages" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-header-chain", "bp-runtime", @@ -1045,7 +1045,7 @@ dependencies = [ [[package]] name = "bp-parachains" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-header-chain", "bp-polkadot-core", @@ -1062,7 +1062,7 @@ dependencies = [ [[package]] name = "bp-polkadot-core" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-messages", "bp-runtime", @@ -1080,7 +1080,7 @@ dependencies = [ [[package]] name = "bp-relayers" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-messages", "bp-runtime", @@ -1094,7 +1094,7 @@ dependencies = [ [[package]] name = "bp-runtime" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -1117,7 +1117,7 @@ dependencies = [ [[package]] name = "bp-test-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-header-chain", "bp-parachains", @@ -1137,7 +1137,7 @@ dependencies = [ [[package]] name = "bp-xcm-bridge-hub" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "sp-std", ] @@ -1145,7 +1145,7 @@ dependencies = [ [[package]] name = "bp-xcm-bridge-hub-router" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -1156,7 +1156,7 @@ dependencies = [ [[package]] name = "bridge-runtime-common" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-header-chain", "bp-messages", @@ -1341,7 +1341,7 @@ dependencies = [ [[package]] name = "ccp-authorities-noting-inherent" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -1368,7 +1368,7 @@ dependencies = [ [[package]] name = "ccp-xcm" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "frame-support", "frame-system", @@ -2299,7 +2299,7 @@ dependencies = [ [[package]] name = "cumulus-client-cli" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "clap", "parity-scale-codec", @@ -2316,7 +2316,7 @@ dependencies = [ [[package]] name = "cumulus-client-collator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-client-consensus-common", "cumulus-client-network", @@ -2339,7 +2339,7 @@ dependencies = [ [[package]] name = "cumulus-client-consensus-aura" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-client-collator", @@ -2381,7 +2381,7 @@ dependencies = [ [[package]] name = "cumulus-client-consensus-common" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-client-pov-recovery", @@ -2410,7 +2410,7 @@ dependencies = [ [[package]] name = "cumulus-client-consensus-proposer" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "anyhow", "async-trait", @@ -2425,7 +2425,7 @@ dependencies = [ [[package]] name = "cumulus-client-network" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-relay-chain-interface", @@ -2448,7 +2448,7 @@ dependencies = [ [[package]] name = "cumulus-client-parachain-inherent" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2472,7 +2472,7 @@ dependencies = [ [[package]] name = "cumulus-client-pov-recovery" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2496,7 +2496,7 @@ dependencies = [ [[package]] name = "cumulus-client-service" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-client-cli", "cumulus-client-collator", @@ -2532,7 +2532,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-dmp-queue" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "frame-benchmarking", @@ -2550,7 +2550,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-parachain-system" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bytes", "cumulus-pallet-parachain-system-proc-macro", @@ -2584,7 +2584,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-parachain-system-proc-macro" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -2595,7 +2595,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-session-benchmarking" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -2609,7 +2609,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-xcm" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -2625,7 +2625,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-xcmp-queue" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bounded-collections", "bp-xcm-bridge-hub-router", @@ -2650,7 +2650,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-aura" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "polkadot-core-primitives", @@ -2664,7 +2664,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-core" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "polkadot-core-primitives", @@ -2681,7 +2681,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-parachain-inherent" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2696,7 +2696,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-proof-size-hostfunction" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "sp-externalities", "sp-runtime-interface", @@ -2706,7 +2706,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-timestamp" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "futures 0.3.30", @@ -2719,7 +2719,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-utility" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -2739,7 +2739,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-inprocess-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2763,7 +2763,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2781,7 +2781,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-minimal-node" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "async-trait", @@ -2822,7 +2822,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-rpc-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2861,7 +2861,7 @@ dependencies = [ [[package]] name = "cumulus-test-relay-sproof-builder" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "parity-scale-codec", @@ -3130,7 +3130,7 @@ dependencies = [ [[package]] name = "dc-orchestrator-chain-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -3349,7 +3349,7 @@ checksum = "9ea835d29036a4087793836fa931b08837ad5e957da9e23886b29586fb9b6650" [[package]] name = "dp-chain-state-snapshot" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "cumulus-primitives-core", "parity-scale-codec", @@ -3361,7 +3361,7 @@ dependencies = [ [[package]] name = "dp-collator-assignment" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -3381,7 +3381,7 @@ dependencies = [ [[package]] name = "dp-core" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -3525,7 +3525,7 @@ dependencies = [ [[package]] name = "emulated-integration-tests-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "asset-test-utils", "bp-messages", @@ -3902,7 +3902,7 @@ dependencies = [ [[package]] name = "fc-api" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "async-trait", "fp-storage", @@ -3914,7 +3914,7 @@ dependencies = [ [[package]] name = "fc-cli" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "clap", "ethereum-types", @@ -3932,7 +3932,7 @@ dependencies = [ [[package]] name = "fc-consensus" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "async-trait", "fp-consensus", @@ -3948,7 +3948,7 @@ dependencies = [ [[package]] name = "fc-db" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "async-trait", "ethereum", @@ -3979,7 +3979,7 @@ dependencies = [ [[package]] name = "fc-mapping-sync" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "fc-db", "fc-storage", @@ -4002,7 +4002,7 @@ dependencies = [ [[package]] name = "fc-rpc" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "ethereum-types", @@ -4057,7 +4057,7 @@ dependencies = [ [[package]] name = "fc-rpc-core" version = "1.1.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "ethereum-types", @@ -4070,7 +4070,7 @@ dependencies = [ [[package]] name = "fc-storage" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "ethereum-types", @@ -4326,7 +4326,7 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "fork-tree" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", ] @@ -4343,7 +4343,7 @@ dependencies = [ [[package]] name = "fp-account" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "hex", "impl-serde", @@ -4362,7 +4362,7 @@ dependencies = [ [[package]] name = "fp-consensus" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "parity-scale-codec", @@ -4374,7 +4374,7 @@ dependencies = [ [[package]] name = "fp-ethereum" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "ethereum-types", @@ -4387,7 +4387,7 @@ dependencies = [ [[package]] name = "fp-evm" version = "3.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "evm", "frame-support", @@ -4403,7 +4403,7 @@ dependencies = [ [[package]] name = "fp-rpc" version = "3.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "ethereum-types", @@ -4420,7 +4420,7 @@ dependencies = [ [[package]] name = "fp-self-contained" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "frame-support", "parity-scale-codec", @@ -4432,7 +4432,7 @@ dependencies = [ [[package]] name = "fp-storage" version = "2.0.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "parity-scale-codec", "serde", @@ -4447,7 +4447,7 @@ checksum = "6c2141d6d6c8512188a7891b4b01590a45f6dac67afb4f255c4124dbb86d4eaa" [[package]] name = "frame-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-support-procedural", @@ -4472,7 +4472,7 @@ dependencies = [ [[package]] name = "frame-benchmarking-cli" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "Inflector", "array-bytes 6.2.2", @@ -4520,7 +4520,7 @@ dependencies = [ [[package]] name = "frame-election-provider-solution-type" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -4531,7 +4531,7 @@ dependencies = [ [[package]] name = "frame-election-provider-support" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-election-provider-solution-type", "frame-support", @@ -4548,7 +4548,7 @@ dependencies = [ [[package]] name = "frame-executive" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -4578,7 +4578,7 @@ dependencies = [ [[package]] name = "frame-remote-externalities" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "indicatif", @@ -4599,7 +4599,7 @@ dependencies = [ [[package]] name = "frame-support" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "aquamarine", "array-bytes 6.2.2", @@ -4640,7 +4640,7 @@ dependencies = [ [[package]] name = "frame-support-procedural" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "Inflector", "cfg-expr", @@ -4659,7 +4659,7 @@ dependencies = [ [[package]] name = "frame-support-procedural-tools" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support-procedural-tools-derive", "proc-macro-crate 3.1.0", @@ -4671,7 +4671,7 @@ dependencies = [ [[package]] name = "frame-support-procedural-tools-derive" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro2", "quote", @@ -4681,7 +4681,7 @@ dependencies = [ [[package]] name = "frame-system" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cfg-if", "docify", @@ -4701,7 +4701,7 @@ dependencies = [ [[package]] name = "frame-system-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -4716,7 +4716,7 @@ dependencies = [ [[package]] name = "frame-system-rpc-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "sp-api", @@ -4725,7 +4725,7 @@ dependencies = [ [[package]] name = "frame-try-runtime" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "parity-scale-codec", @@ -6744,7 +6744,7 @@ dependencies = [ [[package]] name = "mmr-gadget" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "log", @@ -6763,7 +6763,7 @@ dependencies = [ [[package]] name = "mmr-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "anyhow", "jsonrpsee", @@ -7084,7 +7084,7 @@ dependencies = [ [[package]] name = "nimbus-consensus" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "async-backing-primitives", "async-trait", @@ -7124,7 +7124,7 @@ dependencies = [ [[package]] name = "nimbus-primitives" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "async-trait", "frame-benchmarking", @@ -7540,7 +7540,7 @@ dependencies = [ [[package]] name = "pallet-asset-conversion" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7558,7 +7558,7 @@ dependencies = [ [[package]] name = "pallet-asset-rate" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7573,7 +7573,7 @@ dependencies = [ [[package]] name = "pallet-asset-tx-payment" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7591,7 +7591,7 @@ dependencies = [ [[package]] name = "pallet-assets" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7607,7 +7607,7 @@ dependencies = [ [[package]] name = "pallet-async-backing" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-primitives-core", @@ -7627,7 +7627,7 @@ dependencies = [ [[package]] name = "pallet-author-inherent" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "frame-benchmarking", "frame-support", @@ -7707,7 +7707,7 @@ dependencies = [ [[package]] name = "pallet-authority-discovery" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -7737,7 +7737,7 @@ dependencies = [ [[package]] name = "pallet-authorship" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -7751,7 +7751,7 @@ dependencies = [ [[package]] name = "pallet-babe" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7775,7 +7775,7 @@ dependencies = [ [[package]] name = "pallet-bags-list" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "aquamarine", "docify", @@ -7797,7 +7797,7 @@ dependencies = [ [[package]] name = "pallet-balances" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7812,7 +7812,7 @@ dependencies = [ [[package]] name = "pallet-base-fee" version = "1.0.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "fp-evm", "frame-support", @@ -7826,7 +7826,7 @@ dependencies = [ [[package]] name = "pallet-beefy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -7846,7 +7846,7 @@ dependencies = [ [[package]] name = "pallet-beefy-mmr" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "binary-merkle-tree", @@ -7871,7 +7871,7 @@ dependencies = [ [[package]] name = "pallet-bounties" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -7889,7 +7889,7 @@ dependencies = [ [[package]] name = "pallet-bridge-grandpa" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-header-chain", "bp-runtime", @@ -7910,7 +7910,7 @@ dependencies = [ [[package]] name = "pallet-bridge-messages" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-messages", "bp-runtime", @@ -7928,7 +7928,7 @@ dependencies = [ [[package]] name = "pallet-bridge-parachains" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-header-chain", "bp-parachains", @@ -7949,7 +7949,7 @@ dependencies = [ [[package]] name = "pallet-bridge-relayers" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-messages", "bp-relayers", @@ -7969,7 +7969,7 @@ dependencies = [ [[package]] name = "pallet-broker" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "frame-benchmarking", @@ -7986,7 +7986,7 @@ dependencies = [ [[package]] name = "pallet-cc-authorities-noting" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "ccp-authorities-noting-inherent", "cumulus-pallet-parachain-system", @@ -8015,7 +8015,7 @@ dependencies = [ [[package]] name = "pallet-child-bounties" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8066,7 +8066,7 @@ dependencies = [ [[package]] name = "pallet-collator-selection" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8085,7 +8085,7 @@ dependencies = [ [[package]] name = "pallet-collective" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8120,7 +8120,7 @@ dependencies = [ [[package]] name = "pallet-conviction-voting" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "assert_matches", "frame-benchmarking", @@ -8161,7 +8161,7 @@ dependencies = [ [[package]] name = "pallet-democracy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8179,7 +8179,7 @@ dependencies = [ [[package]] name = "pallet-election-provider-multi-phase" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8202,7 +8202,7 @@ dependencies = [ [[package]] name = "pallet-election-provider-support-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8216,7 +8216,7 @@ dependencies = [ [[package]] name = "pallet-elections-phragmen" version = "5.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8235,7 +8235,7 @@ dependencies = [ [[package]] name = "pallet-ethereum" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "ethereum", "ethereum-types", @@ -8258,7 +8258,7 @@ dependencies = [ [[package]] name = "pallet-evm" version = "6.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "environmental", "evm", @@ -8284,7 +8284,7 @@ dependencies = [ [[package]] name = "pallet-evm-chain-id" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "frame-support", "frame-system", @@ -8295,7 +8295,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-balances-erc20" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "fp-evm", "frame-support", @@ -8318,7 +8318,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-batch" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "evm", "fp-evm", @@ -8339,7 +8339,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-call-permit" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "evm", "fp-evm", @@ -8362,7 +8362,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-modexp" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "fp-evm", "num", @@ -8371,7 +8371,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-sha3fips" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "fp-evm", "tiny-keccak", @@ -8380,7 +8380,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-simple" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "fp-evm", "ripemd", @@ -8390,7 +8390,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-xcm-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "fp-evm", "frame-support", @@ -8412,7 +8412,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompileset-assets-erc20" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "fp-evm", "frame-support", @@ -8433,7 +8433,7 @@ dependencies = [ [[package]] name = "pallet-fast-unstake" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "frame-benchmarking", @@ -8452,7 +8452,7 @@ dependencies = [ [[package]] name = "pallet-foreign-asset-creator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "frame-benchmarking", "frame-support", @@ -8471,7 +8471,7 @@ dependencies = [ [[package]] name = "pallet-grandpa" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8494,7 +8494,7 @@ dependencies = [ [[package]] name = "pallet-hotfix-sufficients" version = "1.0.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "frame-benchmarking", "frame-support", @@ -8510,7 +8510,7 @@ dependencies = [ [[package]] name = "pallet-identity" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "enumflags2", "frame-benchmarking", @@ -8527,7 +8527,7 @@ dependencies = [ [[package]] name = "pallet-im-online" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8547,7 +8547,7 @@ dependencies = [ [[package]] name = "pallet-indices" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8624,7 +8624,7 @@ dependencies = [ [[package]] name = "pallet-maintenance-mode" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -8640,7 +8640,7 @@ dependencies = [ [[package]] name = "pallet-membership" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8657,7 +8657,7 @@ dependencies = [ [[package]] name = "pallet-message-queue" version = "7.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "environmental", "frame-benchmarking", @@ -8677,7 +8677,7 @@ dependencies = [ [[package]] name = "pallet-migrations" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "frame-benchmarking", "frame-support", @@ -8696,7 +8696,7 @@ dependencies = [ [[package]] name = "pallet-mmr" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8714,7 +8714,7 @@ dependencies = [ [[package]] name = "pallet-multisig" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8730,7 +8730,7 @@ dependencies = [ [[package]] name = "pallet-nis" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8746,7 +8746,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -8765,7 +8765,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools-benchmarking" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8785,7 +8785,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools-runtime-api" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "pallet-nomination-pools", "parity-scale-codec", @@ -8796,7 +8796,7 @@ dependencies = [ [[package]] name = "pallet-offences" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -8813,7 +8813,7 @@ dependencies = [ [[package]] name = "pallet-offences-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8861,7 +8861,7 @@ dependencies = [ [[package]] name = "pallet-preimage" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8878,7 +8878,7 @@ dependencies = [ [[package]] name = "pallet-proxy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8893,7 +8893,7 @@ dependencies = [ [[package]] name = "pallet-ranked-collective" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8911,7 +8911,7 @@ dependencies = [ [[package]] name = "pallet-recovery" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -8926,7 +8926,7 @@ dependencies = [ [[package]] name = "pallet-referenda" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "assert_matches", "frame-benchmarking", @@ -8978,7 +8978,7 @@ dependencies = [ [[package]] name = "pallet-relay-storage-roots" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-primitives-core", @@ -9001,7 +9001,7 @@ dependencies = [ [[package]] name = "pallet-root-testing" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -9016,7 +9016,7 @@ dependencies = [ [[package]] name = "pallet-scheduler" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "frame-benchmarking", @@ -9054,7 +9054,7 @@ dependencies = [ [[package]] name = "pallet-session" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -9076,7 +9076,7 @@ dependencies = [ [[package]] name = "pallet-session-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9093,7 +9093,7 @@ dependencies = [ [[package]] name = "pallet-society" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9111,7 +9111,7 @@ dependencies = [ [[package]] name = "pallet-staking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -9134,7 +9134,7 @@ dependencies = [ [[package]] name = "pallet-staking-reward-curve" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -9145,7 +9145,7 @@ dependencies = [ [[package]] name = "pallet-staking-reward-fn" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "log", "sp-arithmetic", @@ -9154,7 +9154,7 @@ dependencies = [ [[package]] name = "pallet-staking-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "sp-api", @@ -9164,7 +9164,7 @@ dependencies = [ [[package]] name = "pallet-state-trie-migration" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9205,7 +9205,7 @@ dependencies = [ [[package]] name = "pallet-sudo" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "frame-benchmarking", @@ -9221,7 +9221,7 @@ dependencies = [ [[package]] name = "pallet-timestamp" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "frame-benchmarking", @@ -9241,7 +9241,7 @@ dependencies = [ [[package]] name = "pallet-tips" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9260,7 +9260,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -9276,7 +9276,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "jsonrpsee", "pallet-transaction-payment-rpc-runtime-api", @@ -9292,7 +9292,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment-rpc-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "pallet-transaction-payment", "parity-scale-codec", @@ -9304,7 +9304,7 @@ dependencies = [ [[package]] name = "pallet-treasury" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "frame-benchmarking", @@ -9323,7 +9323,7 @@ dependencies = [ [[package]] name = "pallet-tx-pause" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "frame-benchmarking", @@ -9341,7 +9341,7 @@ dependencies = [ [[package]] name = "pallet-utility" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9357,7 +9357,7 @@ dependencies = [ [[package]] name = "pallet-vesting" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9372,7 +9372,7 @@ dependencies = [ [[package]] name = "pallet-whitelist" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9387,7 +9387,7 @@ dependencies = [ [[package]] name = "pallet-xcm" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bounded-collections", "frame-benchmarking", @@ -9410,7 +9410,7 @@ dependencies = [ [[package]] name = "pallet-xcm-benchmarks" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-benchmarking", "frame-support", @@ -9429,7 +9429,7 @@ dependencies = [ [[package]] name = "pallet-xcm-bridge-hub-router" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bp-xcm-bridge-hub-router", "frame-benchmarking", @@ -9448,7 +9448,7 @@ dependencies = [ [[package]] name = "pallet-xcm-executor-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "frame-benchmarking", "frame-support", @@ -9466,7 +9466,7 @@ dependencies = [ [[package]] name = "parachains-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "cumulus-primitives-utility", @@ -9503,7 +9503,7 @@ dependencies = [ [[package]] name = "parachains-runtimes-test-utils" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "assets-common", "cumulus-pallet-parachain-system", @@ -9856,7 +9856,7 @@ checksum = "626dec3cac7cc0e1577a2ec3fc496277ec2baa084bebad95bb6fdbfae235f84c" [[package]] name = "polkadot-approval-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "futures 0.3.30", @@ -9876,7 +9876,7 @@ dependencies = [ [[package]] name = "polkadot-availability-bitfield-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "always-assert", "futures 0.3.30", @@ -9892,7 +9892,7 @@ dependencies = [ [[package]] name = "polkadot-availability-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "derive_more", "fatality", @@ -9915,7 +9915,7 @@ dependencies = [ [[package]] name = "polkadot-availability-recovery" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "fatality", @@ -9938,7 +9938,7 @@ dependencies = [ [[package]] name = "polkadot-cli" version = "1.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cfg-if", "clap", @@ -9966,7 +9966,7 @@ dependencies = [ [[package]] name = "polkadot-collator-protocol" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "fatality", @@ -9988,7 +9988,7 @@ dependencies = [ [[package]] name = "polkadot-core-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -10000,7 +10000,7 @@ dependencies = [ [[package]] name = "polkadot-dispute-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "derive_more", "fatality", @@ -10025,7 +10025,7 @@ dependencies = [ [[package]] name = "polkadot-erasure-coding" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "polkadot-node-primitives", @@ -10039,7 +10039,7 @@ dependencies = [ [[package]] name = "polkadot-gossip-support" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "futures-timer", @@ -10060,7 +10060,7 @@ dependencies = [ [[package]] name = "polkadot-network-bridge" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "always-assert", "async-trait", @@ -10083,7 +10083,7 @@ dependencies = [ [[package]] name = "polkadot-node-collation-generation" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "parity-scale-codec", @@ -10101,7 +10101,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-approval-voting" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "derive_more", @@ -10134,7 +10134,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-av-store" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "futures 0.3.30", @@ -10156,7 +10156,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-backing" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "fatality", @@ -10175,7 +10175,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-bitfield-signing" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "polkadot-node-subsystem", @@ -10190,7 +10190,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-candidate-validation" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -10211,7 +10211,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-chain-api" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "polkadot-node-metrics", @@ -10225,7 +10225,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-chain-selection" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "futures-timer", @@ -10242,7 +10242,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-dispute-coordinator" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "fatality", "futures 0.3.30", @@ -10261,7 +10261,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-parachains-inherent" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -10278,7 +10278,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-prospective-parachains" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "fatality", @@ -10295,7 +10295,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-provisioner" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "fatality", @@ -10312,7 +10312,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "always-assert", "array-bytes 6.2.2", @@ -10345,7 +10345,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf-checker" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "polkadot-node-primitives", @@ -10361,7 +10361,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cfg-if", "cpu-time", @@ -10386,7 +10386,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-runtime-api" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "polkadot-node-metrics", @@ -10401,7 +10401,7 @@ dependencies = [ [[package]] name = "polkadot-node-jaeger" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "lazy_static", "log", @@ -10419,7 +10419,7 @@ dependencies = [ [[package]] name = "polkadot-node-metrics" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bs58 0.5.0", "futures 0.3.30", @@ -10438,7 +10438,7 @@ dependencies = [ [[package]] name = "polkadot-node-network-protocol" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-channel 1.9.0", "async-trait", @@ -10462,7 +10462,7 @@ dependencies = [ [[package]] name = "polkadot-node-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "bounded-vec", @@ -10485,7 +10485,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "polkadot-node-jaeger", "polkadot-node-subsystem-types", @@ -10495,7 +10495,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-types" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "bitvec", @@ -10523,7 +10523,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-util" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "derive_more", @@ -10558,7 +10558,7 @@ dependencies = [ [[package]] name = "polkadot-overseer" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -10580,7 +10580,7 @@ dependencies = [ [[package]] name = "polkadot-parachain-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bounded-collections", "derive_more", @@ -10597,7 +10597,7 @@ dependencies = [ [[package]] name = "polkadot-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "hex-literal 0.4.1", @@ -10623,7 +10623,7 @@ dependencies = [ [[package]] name = "polkadot-rpc" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "jsonrpsee", "mmr-rpc", @@ -10656,7 +10656,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitvec", "frame-benchmarking", @@ -10708,7 +10708,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-metrics" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bs58 0.5.0", "frame-benchmarking", @@ -10721,7 +10721,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-parachains" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bitflags 1.3.2", "bitvec", @@ -10770,7 +10770,7 @@ dependencies = [ [[package]] name = "polkadot-service" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "frame-benchmarking", @@ -10887,7 +10887,7 @@ dependencies = [ [[package]] name = "polkadot-statement-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "arrayvec 0.7.4", "bitvec", @@ -10910,7 +10910,7 @@ dependencies = [ [[package]] name = "polkadot-statement-table" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "polkadot-primitives", @@ -10991,7 +10991,7 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "precompile-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "environmental", "evm", @@ -11016,7 +11016,7 @@ dependencies = [ [[package]] name = "precompile-utils-macro" version = "0.1.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" dependencies = [ "case", "num_enum", @@ -11684,7 +11684,7 @@ dependencies = [ [[package]] name = "rococo-runtime" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "binary-merkle-tree", "frame-benchmarking", @@ -11780,7 +11780,7 @@ dependencies = [ [[package]] name = "rococo-runtime-constants" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "polkadot-primitives", @@ -12041,7 +12041,7 @@ dependencies = [ [[package]] name = "sc-allocator" version = "4.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "log", "sp-core", @@ -12052,7 +12052,7 @@ dependencies = [ [[package]] name = "sc-authority-discovery" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -12081,7 +12081,7 @@ dependencies = [ [[package]] name = "sc-basic-authorship" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "futures-timer", @@ -12103,7 +12103,7 @@ dependencies = [ [[package]] name = "sc-block-builder" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "sp-api", @@ -12118,7 +12118,7 @@ dependencies = [ [[package]] name = "sc-chain-spec" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "docify", @@ -12143,7 +12143,7 @@ dependencies = [ [[package]] name = "sc-chain-spec-derive" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -12154,7 +12154,7 @@ dependencies = [ [[package]] name = "sc-cli" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "bip39", @@ -12198,7 +12198,7 @@ dependencies = [ [[package]] name = "sc-client-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "fnv", "futures 0.3.30", @@ -12225,7 +12225,7 @@ dependencies = [ [[package]] name = "sc-client-db" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "hash-db", "kvdb", @@ -12251,7 +12251,7 @@ dependencies = [ [[package]] name = "sc-consensus" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -12276,7 +12276,7 @@ dependencies = [ [[package]] name = "sc-consensus-aura" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -12305,7 +12305,7 @@ dependencies = [ [[package]] name = "sc-consensus-babe" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "fork-tree", @@ -12340,7 +12340,7 @@ dependencies = [ [[package]] name = "sc-consensus-babe-rpc" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "jsonrpsee", @@ -12362,7 +12362,7 @@ dependencies = [ [[package]] name = "sc-consensus-beefy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12397,7 +12397,7 @@ dependencies = [ [[package]] name = "sc-consensus-beefy-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "jsonrpsee", @@ -12416,7 +12416,7 @@ dependencies = [ [[package]] name = "sc-consensus-epochs" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "fork-tree", "parity-scale-codec", @@ -12429,7 +12429,7 @@ dependencies = [ [[package]] name = "sc-consensus-grandpa" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ahash 0.8.8", "array-bytes 6.2.2", @@ -12471,7 +12471,7 @@ dependencies = [ [[package]] name = "sc-consensus-grandpa-rpc" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "finality-grandpa", "futures 0.3.30", @@ -12491,7 +12491,7 @@ dependencies = [ [[package]] name = "sc-consensus-manual-seal" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "assert_matches", "async-trait", @@ -12526,7 +12526,7 @@ dependencies = [ [[package]] name = "sc-consensus-slots" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -12549,7 +12549,7 @@ dependencies = [ [[package]] name = "sc-executor" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "log", "parity-scale-codec", @@ -12572,7 +12572,7 @@ dependencies = [ [[package]] name = "sc-executor-common" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "sc-allocator", @@ -12585,7 +12585,7 @@ dependencies = [ [[package]] name = "sc-executor-wasmtime" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "anyhow", "cfg-if", @@ -12605,7 +12605,7 @@ dependencies = [ [[package]] name = "sc-informant" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ansi_term", "futures 0.3.30", @@ -12622,7 +12622,7 @@ dependencies = [ [[package]] name = "sc-keystore" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "parking_lot 0.12.1", @@ -12636,7 +12636,7 @@ dependencies = [ [[package]] name = "sc-mixnet" version = "0.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 4.2.0", "arrayvec 0.7.4", @@ -12665,7 +12665,7 @@ dependencies = [ [[package]] name = "sc-network" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12708,7 +12708,7 @@ dependencies = [ [[package]] name = "sc-network-bitswap" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-channel 1.9.0", "cid", @@ -12728,7 +12728,7 @@ dependencies = [ [[package]] name = "sc-network-common" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "bitflags 1.3.2", @@ -12745,7 +12745,7 @@ dependencies = [ [[package]] name = "sc-network-gossip" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ahash 0.8.8", "futures 0.3.30", @@ -12764,7 +12764,7 @@ dependencies = [ [[package]] name = "sc-network-light" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12785,7 +12785,7 @@ dependencies = [ [[package]] name = "sc-network-sync" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12821,7 +12821,7 @@ dependencies = [ [[package]] name = "sc-network-test" version = "0.8.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -12852,7 +12852,7 @@ dependencies = [ [[package]] name = "sc-network-transactions" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "futures 0.3.30", @@ -12871,7 +12871,7 @@ dependencies = [ [[package]] name = "sc-offchain" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "bytes", @@ -12905,7 +12905,7 @@ dependencies = [ [[package]] name = "sc-proposer-metrics" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "log", "substrate-prometheus-endpoint", @@ -12914,7 +12914,7 @@ dependencies = [ [[package]] name = "sc-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "jsonrpsee", @@ -12946,7 +12946,7 @@ dependencies = [ [[package]] name = "sc-rpc-api" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -12966,7 +12966,7 @@ dependencies = [ [[package]] name = "sc-rpc-server" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "http", "jsonrpsee", @@ -12981,7 +12981,7 @@ dependencies = [ [[package]] name = "sc-rpc-spec-v2" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "futures 0.3.30", @@ -13010,7 +13010,7 @@ dependencies = [ [[package]] name = "sc-service" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "directories", @@ -13073,7 +13073,7 @@ dependencies = [ [[package]] name = "sc-state-db" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "log", "parity-scale-codec", @@ -13084,7 +13084,7 @@ dependencies = [ [[package]] name = "sc-storage-monitor" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "clap", "fs4", @@ -13097,7 +13097,7 @@ dependencies = [ [[package]] name = "sc-sync-state-rpc" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -13116,7 +13116,7 @@ dependencies = [ [[package]] name = "sc-sysinfo" version = "6.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "derive_more", "futures 0.3.30", @@ -13136,7 +13136,7 @@ dependencies = [ [[package]] name = "sc-telemetry" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "chrono", "futures 0.3.30", @@ -13155,7 +13155,7 @@ dependencies = [ [[package]] name = "sc-tracing" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ansi_term", "chrono", @@ -13185,7 +13185,7 @@ dependencies = [ [[package]] name = "sc-tracing-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -13196,7 +13196,7 @@ dependencies = [ [[package]] name = "sc-transaction-pool" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -13222,7 +13222,7 @@ dependencies = [ [[package]] name = "sc-transaction-pool-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -13238,7 +13238,7 @@ dependencies = [ [[package]] name = "sc-utils" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-channel 1.9.0", "futures 0.3.30", @@ -13693,7 +13693,7 @@ dependencies = [ [[package]] name = "slot-range-helper" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "enumn", "parity-scale-codec", @@ -13887,7 +13887,7 @@ dependencies = [ [[package]] name = "sp-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "hash-db", "log", @@ -13908,7 +13908,7 @@ dependencies = [ [[package]] name = "sp-api-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "Inflector", "blake2 0.10.6", @@ -13922,7 +13922,7 @@ dependencies = [ [[package]] name = "sp-application-crypto" version = "23.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -13935,7 +13935,7 @@ dependencies = [ [[package]] name = "sp-arithmetic" version = "16.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "integer-sqrt", "num-traits", @@ -13949,7 +13949,7 @@ dependencies = [ [[package]] name = "sp-authority-discovery" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -13962,7 +13962,7 @@ dependencies = [ [[package]] name = "sp-block-builder" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "sp-api", "sp-inherents", @@ -13973,7 +13973,7 @@ dependencies = [ [[package]] name = "sp-blockchain" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "log", @@ -13991,7 +13991,7 @@ dependencies = [ [[package]] name = "sp-consensus" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "futures 0.3.30", @@ -14006,7 +14006,7 @@ dependencies = [ [[package]] name = "sp-consensus-aura" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "parity-scale-codec", @@ -14023,7 +14023,7 @@ dependencies = [ [[package]] name = "sp-consensus-babe" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "parity-scale-codec", @@ -14042,7 +14042,7 @@ dependencies = [ [[package]] name = "sp-consensus-beefy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "lazy_static", "parity-scale-codec", @@ -14061,7 +14061,7 @@ dependencies = [ [[package]] name = "sp-consensus-grandpa" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "finality-grandpa", "log", @@ -14079,7 +14079,7 @@ dependencies = [ [[package]] name = "sp-consensus-slots" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -14091,7 +14091,7 @@ dependencies = [ [[package]] name = "sp-core" version = "21.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "bip39", @@ -14136,7 +14136,7 @@ dependencies = [ [[package]] name = "sp-core-hashing" version = "9.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "blake2b_simd", "byteorder", @@ -14149,7 +14149,7 @@ dependencies = [ [[package]] name = "sp-core-hashing-proc-macro" version = "9.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "quote", "sp-core-hashing", @@ -14159,7 +14159,7 @@ dependencies = [ [[package]] name = "sp-database" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "kvdb", "parking_lot 0.12.1", @@ -14168,7 +14168,7 @@ dependencies = [ [[package]] name = "sp-debug-derive" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "proc-macro2", "quote", @@ -14178,7 +14178,7 @@ dependencies = [ [[package]] name = "sp-externalities" version = "0.19.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "environmental", "parity-scale-codec", @@ -14189,7 +14189,7 @@ dependencies = [ [[package]] name = "sp-genesis-builder" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "serde_json", "sp-api", @@ -14200,7 +14200,7 @@ dependencies = [ [[package]] name = "sp-inherents" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "impl-trait-for-tuples", @@ -14214,7 +14214,7 @@ dependencies = [ [[package]] name = "sp-io" version = "23.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bytes", "ed25519-dalek", @@ -14238,7 +14238,7 @@ dependencies = [ [[package]] name = "sp-keyring" version = "24.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "sp-core", "sp-runtime", @@ -14248,7 +14248,7 @@ dependencies = [ [[package]] name = "sp-keystore" version = "0.27.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "parking_lot 0.12.1", @@ -14260,7 +14260,7 @@ dependencies = [ [[package]] name = "sp-maybe-compressed-blob" version = "4.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "thiserror", "zstd 0.12.4", @@ -14269,7 +14269,7 @@ dependencies = [ [[package]] name = "sp-metadata-ir" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-metadata", "parity-scale-codec", @@ -14280,7 +14280,7 @@ dependencies = [ [[package]] name = "sp-mixnet" version = "0.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -14292,7 +14292,7 @@ dependencies = [ [[package]] name = "sp-mmr-primitives" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ckb-merkle-mountain-range", "log", @@ -14310,7 +14310,7 @@ dependencies = [ [[package]] name = "sp-npos-elections" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -14324,7 +14324,7 @@ dependencies = [ [[package]] name = "sp-offchain" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "sp-api", "sp-core", @@ -14334,7 +14334,7 @@ dependencies = [ [[package]] name = "sp-panic-handler" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "backtrace", "lazy_static", @@ -14344,7 +14344,7 @@ dependencies = [ [[package]] name = "sp-rpc" version = "6.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "rustc-hash", "serde", @@ -14354,7 +14354,7 @@ dependencies = [ [[package]] name = "sp-runtime" version = "24.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "docify", "either", @@ -14378,7 +14378,7 @@ dependencies = [ [[package]] name = "sp-runtime-interface" version = "17.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bytes", "impl-trait-for-tuples", @@ -14396,7 +14396,7 @@ dependencies = [ [[package]] name = "sp-runtime-interface-proc-macro" version = "11.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "Inflector", "expander 2.0.0", @@ -14409,7 +14409,7 @@ dependencies = [ [[package]] name = "sp-session" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "scale-info", @@ -14424,7 +14424,7 @@ dependencies = [ [[package]] name = "sp-staking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "impl-trait-for-tuples", "parity-scale-codec", @@ -14438,7 +14438,7 @@ dependencies = [ [[package]] name = "sp-state-machine" version = "0.28.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "hash-db", "log", @@ -14459,7 +14459,7 @@ dependencies = [ [[package]] name = "sp-statement-store" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "aes-gcm", "curve25519-dalek 4.1.2", @@ -14483,12 +14483,12 @@ dependencies = [ [[package]] name = "sp-std" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" [[package]] name = "sp-storage" version = "13.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "impl-serde", "parity-scale-codec", @@ -14501,7 +14501,7 @@ dependencies = [ [[package]] name = "sp-timestamp" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "parity-scale-codec", @@ -14514,7 +14514,7 @@ dependencies = [ [[package]] name = "sp-tracing" version = "10.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "sp-std", @@ -14526,7 +14526,7 @@ dependencies = [ [[package]] name = "sp-transaction-pool" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "sp-api", "sp-runtime", @@ -14535,7 +14535,7 @@ dependencies = [ [[package]] name = "sp-transaction-storage-proof" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "parity-scale-codec", @@ -14550,7 +14550,7 @@ dependencies = [ [[package]] name = "sp-trie" version = "22.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ahash 0.8.8", "hash-db", @@ -14574,7 +14574,7 @@ dependencies = [ [[package]] name = "sp-version" version = "22.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "impl-serde", "parity-scale-codec", @@ -14591,7 +14591,7 @@ dependencies = [ [[package]] name = "sp-version-proc-macro" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "parity-scale-codec", "proc-macro2", @@ -14602,7 +14602,7 @@ dependencies = [ [[package]] name = "sp-wasm-interface" version = "14.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "anyhow", "impl-trait-for-tuples", @@ -14615,7 +14615,7 @@ dependencies = [ [[package]] name = "sp-weights" version = "20.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "bounded-collections", "parity-scale-codec", @@ -14810,7 +14810,7 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "staging-parachain-info" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -14824,7 +14824,7 @@ dependencies = [ [[package]] name = "staging-xcm" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "bounded-collections", @@ -14842,7 +14842,7 @@ dependencies = [ [[package]] name = "staging-xcm-builder" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "frame-system", @@ -14864,7 +14864,7 @@ dependencies = [ [[package]] name = "staging-xcm-executor" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "environmental", "frame-benchmarking", @@ -14992,12 +14992,12 @@ dependencies = [ [[package]] name = "substrate-build-script-utils" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" [[package]] name = "substrate-frame-rpc-system" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-system-rpc-runtime-api", "futures 0.3.30", @@ -15016,7 +15016,7 @@ dependencies = [ [[package]] name = "substrate-prometheus-endpoint" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "hyper", "log", @@ -15028,7 +15028,7 @@ dependencies = [ [[package]] name = "substrate-rpc-client" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "jsonrpsee", @@ -15041,7 +15041,7 @@ dependencies = [ [[package]] name = "substrate-state-trie-migration-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -15058,7 +15058,7 @@ dependencies = [ [[package]] name = "substrate-test-client" version = "2.0.1" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "async-trait", @@ -15084,7 +15084,7 @@ dependencies = [ [[package]] name = "substrate-test-runtime" version = "2.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "array-bytes 6.2.2", "frame-executive", @@ -15125,7 +15125,7 @@ dependencies = [ [[package]] name = "substrate-test-runtime-client" version = "2.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "futures 0.3.30", "sc-block-builder", @@ -15143,7 +15143,7 @@ dependencies = [ [[package]] name = "substrate-wasm-builder" version = "5.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "ansi_term", "build-helper", @@ -15431,7 +15431,7 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "test-relay-sproof-builder" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#e77a98ee860d9a2e8492b65e3a30257b0d57c39d" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#2d02dd7c338a8b567fd9f714c0b7a0f088052703" dependencies = [ "cumulus-primitives-core", "dp-collator-assignment", @@ -15943,7 +15943,7 @@ dependencies = [ [[package]] name = "tracing-gum" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "coarsetime", "polkadot-primitives", @@ -15954,7 +15954,7 @@ dependencies = [ [[package]] name = "tracing-gum-proc-macro" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "expander 2.0.0", "proc-macro-crate 3.1.0", @@ -16084,7 +16084,7 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "try-runtime-cli" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "async-trait", "clap", @@ -16768,7 +16768,7 @@ checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" [[package]] name = "westend-runtime" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "binary-merkle-tree", "bitvec", @@ -16874,7 +16874,7 @@ dependencies = [ [[package]] name = "westend-runtime-constants" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "frame-support", "polkadot-primitives", @@ -17253,7 +17253,7 @@ dependencies = [ [[package]] name = "xcm-emulator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-pallet-xcmp-queue", @@ -17288,7 +17288,7 @@ dependencies = [ [[package]] name = "xcm-primitives" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" dependencies = [ "sp-runtime", ] @@ -17296,7 +17296,7 @@ dependencies = [ [[package]] name = "xcm-procedural" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#fb422ea2572811a4d339dc100cb32d6aa63fe0c9" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" dependencies = [ "Inflector", "proc-macro2", diff --git a/Cargo.toml b/Cargo.toml index bf7d5bf1c..d36e7fea6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -67,204 +67,204 @@ tp-maths = { path = "primitives/maths", default-features = false } tp-traits = { path = "primitives/traits", default-features = false } # Dancekit (wasm) -ccp-authorities-noting-inherent = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -ccp-xcm = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -dp-chain-state-snapshot = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -dp-collator-assignment = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -dp-core = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-cc-authorities-noting = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-xcm-executor-utils = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -test-relay-sproof-builder = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +ccp-authorities-noting-inherent = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +ccp-xcm = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +dp-chain-state-snapshot = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +dp-collator-assignment = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +dp-core = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-cc-authorities-noting = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-xcm-executor-utils = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +test-relay-sproof-builder = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Dancekit (client) -dc-orchestrator-chain-interface = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0" } +dc-orchestrator-chain-interface = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead" } # Moonkit (wasm) -async-backing-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -nimbus-consensus = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0" } -nimbus-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-async-backing = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-author-inherent = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-balances-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-batch = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-call-permit = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-xcm-utils = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompileset-assets-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-foreign-asset-creator = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-maintenance-mode = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-migrations = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-relay-storage-roots = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -xcm-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +async-backing-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +nimbus-consensus = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead" } +nimbus-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-async-backing = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-author-inherent = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-balances-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-batch = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-call-permit = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-xcm-utils = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompileset-assets-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-foreign-asset-creator = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-maintenance-mode = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-migrations = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-relay-storage-roots = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +xcm-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Substrate (wasm) -frame-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -frame-executive = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -frame-support = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "4.0.0-dev", default-features = false } -frame-system = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "4.0.0-dev", default-features = false } -frame-system-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -frame-system-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -frame-try-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-asset-rate = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-assets = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-balances = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-identity = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-message-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-multisig = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-proxy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-root-testing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-sudo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-transaction-payment = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-treasury = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-tx-pause = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +frame-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +frame-executive = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +frame-support = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "4.0.0-dev", default-features = false } +frame-system = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "4.0.0-dev", default-features = false } +frame-system-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +frame-system-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +frame-try-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-asset-rate = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-assets = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-balances = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-identity = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-message-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-multisig = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-proxy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-root-testing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-sudo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-transaction-payment = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-treasury = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-tx-pause = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } parity-scale-codec = { version = "3.0.0", default-features = false, features = [ "derive", "max-encoded-len" ] } scale-info = { version = "2.10.0", default-features = false } -sp-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-application-crypto = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-consensus-babe = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-consensus-beefy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-core = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "21.0.0", default-features = false } -sp-debug-derive = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-inherents = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-io = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "23.0.0", default-features = false } -sp-keyring = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "24.0.0", default-features = false } -sp-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "24.0.0", default-features = false } -sp-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-state-machine = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-std = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-trie = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-version = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-application-crypto = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-consensus-babe = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-consensus-beefy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-core = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "21.0.0", default-features = false } +sp-debug-derive = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-inherents = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-io = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "23.0.0", default-features = false } +sp-keyring = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "24.0.0", default-features = false } +sp-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "24.0.0", default-features = false } +sp-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-state-machine = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-std = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-trie = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-version = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Substrate (client) -frame-benchmarking-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -pallet-transaction-payment-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-transaction-payment-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sc-basic-authorship = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-chain-spec = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-client-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-consensus-grandpa = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-consensus-manual-seal = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-network-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-network-sync = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-network-test = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-network-transactions = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-sysinfo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-telemetry = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-tracing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-transaction-pool-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sc-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sp-blockchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -sp-externalities = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-genesis-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-storage = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -sp-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -substrate-build-script-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -substrate-frame-rpc-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -substrate-prometheus-endpoint = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -substrate-test-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -substrate-test-runtime-client = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -substrate-wasm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -try-runtime-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +frame-benchmarking-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +pallet-transaction-payment-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-transaction-payment-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sc-basic-authorship = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-chain-spec = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-client-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-consensus-grandpa = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-consensus-manual-seal = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-network-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-network-sync = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-network-test = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-network-transactions = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-sysinfo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-telemetry = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-tracing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-transaction-pool-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sc-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sp-blockchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +sp-externalities = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-genesis-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-storage = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +substrate-build-script-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +substrate-frame-rpc-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +substrate-prometheus-endpoint = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +substrate-test-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +substrate-test-runtime-client = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +substrate-wasm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +try-runtime-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } # Polkadot (wasm) -pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-xcm-benchmarks = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -polkadot-core-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -polkadot-node-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -polkadot-parachain-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -polkadot-runtime-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -polkadot-runtime-parachains = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -staging-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -staging-xcm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -staging-xcm-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -westend-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -westend-runtime-constants = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-xcm-benchmarks = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +polkadot-core-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +polkadot-node-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +polkadot-parachain-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +polkadot-runtime-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +polkadot-runtime-parachains = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +staging-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +staging-xcm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +staging-xcm-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +westend-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +westend-runtime-constants = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Polkadot (client) -polkadot-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -polkadot-node-subsystem = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -polkadot-overseer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -polkadot-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -polkadot-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +polkadot-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +polkadot-node-subsystem = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +polkadot-overseer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +polkadot-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +polkadot-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } # Cumulus (wasm) -cumulus-pallet-dmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-pallet-parachain-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false, features = [ "parameterized-consensus-hook" ] } -cumulus-pallet-session-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-pallet-xcmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-primitives-core = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-primitives-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-primitives-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -parachain-info = { package = "staging-parachain-info", git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -parachains-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-pallet-dmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-pallet-parachain-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false, features = [ "parameterized-consensus-hook" ] } +cumulus-pallet-session-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-pallet-xcmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-primitives-core = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-primitives-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-primitives-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +parachain-info = { package = "staging-parachain-info", git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +parachains-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Cumulus (client) -assets-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -cumulus-client-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-collator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-consensus-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-consensus-proposer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-pov-recovery = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-client-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-primitives-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-relay-chain-interface = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -cumulus-test-relay-sproof-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } -emulated-integration-tests-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } -xcm-emulator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +assets-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +cumulus-client-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-collator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-consensus-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-consensus-proposer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-pov-recovery = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-client-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-primitives-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-relay-chain-interface = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-test-relay-sproof-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +emulated-integration-tests-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +xcm-emulator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Frontier (wasm) -fp-account = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fp-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fp-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fp-self-contained = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-base-fee = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-ethereum = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-chain-id = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-modexp = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-sha3fips = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-evm-precompile-simple = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -pallet-hotfix-sufficients = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -precompile-utils = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fp-account = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fp-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fp-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fp-self-contained = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-base-fee = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-ethereum = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-chain-id = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-modexp = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-sha3fips = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-evm-precompile-simple = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-hotfix-sufficients = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +precompile-utils = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # Frontier (client) -fc-api = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fc-cli = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fc-consensus = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fc-db = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fc-mapping-sync = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fc-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", features = [ +fc-api = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-cli = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-consensus = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-db = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-mapping-sync = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", features = [ "rpc-binary-search-estimate", ] } -fc-rpc-core = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } -fc-storage = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-rpc-core = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-storage = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } # General (wasm) bounded-collections = { version = "0.1.8", default-features = false } From 4b79ce0415ccf43be318f03bfa8598d3cfef6602 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Mon, 11 Mar 2024 12:57:45 -0700 Subject: [PATCH 13/35] fix rust tests --- runtime/dancebox/tests/common/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runtime/dancebox/tests/common/mod.rs b/runtime/dancebox/tests/common/mod.rs index d0d04a97a..817550eca 100644 --- a/runtime/dancebox/tests/common/mod.rs +++ b/runtime/dancebox/tests/common/mod.rs @@ -150,7 +150,7 @@ pub fn set_parachain_inherent_data() { let relay_sproof = RelayStateSproofBuilder { para_id: 100u32.into(), included_para_head: Some(HeadData(vec![1, 2, 3])), - current_slot: (current_slot() * 2).into(), + current_slot: (current_slot()).into(), ..Default::default() }; @@ -192,7 +192,7 @@ pub fn set_parachain_inherent_data_random_seed(random_seed: [u8; 32]) { sproof.para_id = 100u32.into(); sproof.included_para_head = Some(HeadData(vec![1, 2, 3])); - sproof.current_slot = (slot * 2).into(); + sproof.current_slot = (slot).into(); sproof.into_state_root_and_proof() }; From f2b934c6b354ffb5a305ab53f45681ce7207019e Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 12 Mar 2024 19:39:40 -0700 Subject: [PATCH 14/35] add end_lookahead channel --- client/consensus/Cargo.toml | 2 +- client/consensus/src/collators/lookahead.rs | 22 ++++++++++++++++++++- node/src/container_chain_spawner.rs | 10 ++++++---- node/src/service.rs | 10 ++++++++-- 4 files changed, 36 insertions(+), 8 deletions(-) diff --git a/client/consensus/Cargo.toml b/client/consensus/Cargo.toml index 45c06a726..5121d63ae 100644 --- a/client/consensus/Cargo.toml +++ b/client/consensus/Cargo.toml @@ -65,6 +65,7 @@ futures = { workspace = true } log = { workspace = true } parity-scale-codec = { workspace = true, features = [ "derive" ] } tracing = { workspace = true } +tokio = { workspace = true } [dev-dependencies] cumulus-test-relay-sproof-builder = { workspace = true } @@ -78,4 +79,3 @@ sc-network-test = { workspace = true } sp-keyring = { workspace = true } substrate-test-runtime-client = { workspace = true } tempfile = { workspace = true } -tokio = { workspace = true } diff --git a/client/consensus/src/collators/lookahead.rs b/client/consensus/src/collators/lookahead.rs index 35d0f2e74..ac24a01e4 100644 --- a/client/consensus/src/collators/lookahead.rs +++ b/client/consensus/src/collators/lookahead.rs @@ -61,8 +61,8 @@ use sp_core::crypto::Pair; use sp_inherents::CreateInherentDataProviders; use sp_keystore::KeystorePtr; use sp_runtime::traits::{Block as BlockT, Header as HeaderT, Member}; -//use sp_timestamp::Timestamp; use std::{convert::TryFrom, error::Error, sync::Arc, time::Duration}; +use tokio::sync::watch::Receiver; use crate::{ collators::{self as collator_util, tanssi_claim_slot, SlotClaim}, @@ -90,6 +90,7 @@ pub struct Params>, } /// Run async-backing-friendly for Tanssi Aura. @@ -177,7 +178,26 @@ where collator_util::Collator::::new(params) }; + // If we move forward without marking the value as unchanged, + // the channel will assume that the value has already changed + // in a different step than the one we want (inside containerChainSpawner) + // and will not kill the lookahead collator properly if it was already running. + if let Some(end_lookahead_receiver) = &mut params.end_lookahead_receiver { + end_lookahead_receiver.mark_unchanged(); + } + while let Some(relay_parent_header) = import_notifications.next().await { + if let Some(end_lookahead_receiver) = &mut params.end_lookahead_receiver { + // If the value of the channel has changed, it means that + // containerChainSpawner has informed that we need to tear down + // this consensus task, meaning that we don't need to spawn the + // lookahead collator twice. + if let Ok(true) = end_lookahead_receiver.has_changed() { + log::info!("Lookahead collator was already running! Exiting..."); + return; + } + } + let relay_parent = relay_parent_header.hash(); if !is_para_scheduled(relay_parent, params.para_id, &mut params.overseer_handle).await { diff --git a/node/src/container_chain_spawner.rs b/node/src/container_chain_spawner.rs index b1c9eae63..ef6f79bed 100644 --- a/node/src/container_chain_spawner.rs +++ b/node/src/container_chain_spawner.rs @@ -51,7 +51,7 @@ use { time::Instant, }, tokio::{ - sync::{mpsc, oneshot}, + sync::{mpsc, oneshot, watch}, time::{sleep, Duration}, }, }; @@ -411,11 +411,11 @@ impl ContainerChainSpawner { } /// Receive and process `CcSpawnMsg`s indefinitely - pub async fn rx_loop(self, mut rx: mpsc::UnboundedReceiver) { + pub async fn rx_loop(self, mut rx: mpsc::UnboundedReceiver, end_lookahead_sender: watch::Sender<()>) { while let Some(msg) = rx.recv().await { match msg { CcSpawnMsg::UpdateAssignment { current, next } => { - self.handle_update_assignment(current, next).await; + self.handle_update_assignment(current, next, &end_lookahead_sender).await; } } } @@ -427,7 +427,7 @@ impl ContainerChainSpawner { } /// Handle `CcSpawnMsg::UpdateAssignment` - async fn handle_update_assignment(&self, current: Option, next: Option) { + async fn handle_update_assignment(&self, current: Option, next: Option, end_lookahead_sender: &watch::Sender<()>) { let HandleUpdateAssignmentResult { call_collate_on, chains_to_stop, @@ -443,6 +443,8 @@ impl ContainerChainSpawner { // Call collate_on, to start collation on a chain that was already running before if let Some(f) = call_collate_on { + // End previous tanssi-aura job + let _ = end_lookahead_sender.send(()); f(); } diff --git a/node/src/service.rs b/node/src/service.rs index 5f508c281..7a85dd393 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -79,7 +79,7 @@ use { //collators::basic::{self as basic_tanssi_aura, Params as BasicTanssiAuraParams}, OrchestratorAuraWorkerAuxData, }, - tokio::sync::mpsc::{unbounded_channel, UnboundedSender}, + tokio::sync::{watch::Receiver, mpsc::{unbounded_channel, UnboundedSender}}, }; type FullBackend = TFullBackend; @@ -401,6 +401,8 @@ async fn start_node_impl( sync_service: node_builder.network.sync_service.clone(), })?; + let (end_lookahead_sender, end_lookahead_receiver) = tokio::sync::watch::channel(()); + if validator { let collator_key = collator_key .clone() @@ -452,6 +454,7 @@ async fn start_node_impl( overseer.clone(), announce_block.clone(), proposer_factory.clone(), + Some(end_lookahead_receiver.clone()), ) } }; @@ -511,7 +514,7 @@ async fn start_node_impl( node_builder.task_manager.spawn_essential_handle().spawn( "container-chain-spawner-rx-loop", None, - container_chain_spawner.rx_loop(cc_spawn_rx), + container_chain_spawner.rx_loop(cc_spawn_rx, end_lookahead_sender), ); node_builder.task_manager.spawn_essential_handle().spawn( @@ -832,6 +835,7 @@ fn start_consensus_container( authoring_duration: Duration::from_millis(500), para_backend: backend, code_hash_provider, + end_lookahead_receiver: None, //collation_request_receiver: None, }; @@ -854,6 +858,7 @@ fn start_consensus_orchestrator( overseer_handle: OverseerHandle, announce_block: Arc>) + Send + Sync>, proposer_factory: ParachainProposerFactory, + end_lookahead_receiver: Option> ) { let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client) .expect("start_consensus_orchestrator: slot duration should exist"); @@ -962,6 +967,7 @@ fn start_consensus_orchestrator( authoring_duration: Duration::from_millis(500), code_hash_provider, para_backend: backend, + end_lookahead_receiver, //collation_request_receiver: None, }; From 97e80e9f2ac80a7a1421beb4e915581187411cb9 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 12 Mar 2024 19:41:30 -0700 Subject: [PATCH 15/35] changes in zombie test --- test/suites/para/test_tanssi_containers.ts | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/test/suites/para/test_tanssi_containers.ts b/test/suites/para/test_tanssi_containers.ts index 8fad045f4..fb2251dce 100644 --- a/test/suites/para/test_tanssi_containers.ts +++ b/test/suites/para/test_tanssi_containers.ts @@ -310,7 +310,7 @@ describeSuite({ it({ id: "T15", title: "Deregister container chain 2002, collators should move to tanssi", - timeout: 600000, + timeout: 300000, test: async function () { const keyring = new Keyring({ type: "sr25519" }); const alice = keyring.addFromUri("//Alice", { name: "Alice default" }); @@ -346,7 +346,7 @@ describeSuite({ test: async function () { // This test depends on T12 and T15 to set blockNumber2002Start and blockNumber2002End // TODO: don't hardcode the period here - const sessionPeriod = 5; + const sessionPeriod = 10; // The block range must start and end on session boundaries expect(blockNumber2002Start % sessionPeriod).to.be.equal(0); expect(blockNumber2002End % sessionPeriod).to.be.equal(0); @@ -358,8 +358,8 @@ describeSuite({ // TODO: this passes if only 2 authors are creating blocks, think a way to test that case await countUniqueBlockAuthors(paraApi, blockNumber, blockNumber2002Start - 1, 4); - expect(sessionPeriod * 10 < blockNumber2002End, "2002 should have deregistered after first rotation"); - expect(sessionPeriod * 20 > blockNumber2002End, "2002 should have deregistered before second rotation"); + expect(sessionPeriod * 5 < blockNumber2002End, "2002 should have deregistered after first rotation"); + expect(sessionPeriod * 10 > blockNumber2002End, "2002 should have deregistered before second rotation"); // While 2002 is live: 2 authors (the other 2 went to container chain 2002) // We take from the first block that rotates, otherwise rotation kicks in From 414ef9db1c6e55909c8497134728e9a7e1e591ef Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 12 Mar 2024 19:42:42 -0700 Subject: [PATCH 16/35] fmt --- client/consensus/src/collators/lookahead.rs | 2 +- node/src/container_chain_spawner.rs | 16 +++++++++++++--- node/src/service.rs | 7 +++++-- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/client/consensus/src/collators/lookahead.rs b/client/consensus/src/collators/lookahead.rs index ac24a01e4..780dad247 100644 --- a/client/consensus/src/collators/lookahead.rs +++ b/client/consensus/src/collators/lookahead.rs @@ -188,7 +188,7 @@ where while let Some(relay_parent_header) = import_notifications.next().await { if let Some(end_lookahead_receiver) = &mut params.end_lookahead_receiver { - // If the value of the channel has changed, it means that + // If the value of the channel has changed, it means that // containerChainSpawner has informed that we need to tear down // this consensus task, meaning that we don't need to spawn the // lookahead collator twice. diff --git a/node/src/container_chain_spawner.rs b/node/src/container_chain_spawner.rs index ef6f79bed..e266a4d65 100644 --- a/node/src/container_chain_spawner.rs +++ b/node/src/container_chain_spawner.rs @@ -411,11 +411,16 @@ impl ContainerChainSpawner { } /// Receive and process `CcSpawnMsg`s indefinitely - pub async fn rx_loop(self, mut rx: mpsc::UnboundedReceiver, end_lookahead_sender: watch::Sender<()>) { + pub async fn rx_loop( + self, + mut rx: mpsc::UnboundedReceiver, + end_lookahead_sender: watch::Sender<()>, + ) { while let Some(msg) = rx.recv().await { match msg { CcSpawnMsg::UpdateAssignment { current, next } => { - self.handle_update_assignment(current, next, &end_lookahead_sender).await; + self.handle_update_assignment(current, next, &end_lookahead_sender) + .await; } } } @@ -427,7 +432,12 @@ impl ContainerChainSpawner { } /// Handle `CcSpawnMsg::UpdateAssignment` - async fn handle_update_assignment(&self, current: Option, next: Option, end_lookahead_sender: &watch::Sender<()>) { + async fn handle_update_assignment( + &self, + current: Option, + next: Option, + end_lookahead_sender: &watch::Sender<()>, + ) { let HandleUpdateAssignmentResult { call_collate_on, chains_to_stop, diff --git a/node/src/service.rs b/node/src/service.rs index 7a85dd393..3cf6e6208 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -79,7 +79,10 @@ use { //collators::basic::{self as basic_tanssi_aura, Params as BasicTanssiAuraParams}, OrchestratorAuraWorkerAuxData, }, - tokio::sync::{watch::Receiver, mpsc::{unbounded_channel, UnboundedSender}}, + tokio::sync::{ + mpsc::{unbounded_channel, UnboundedSender}, + watch::Receiver, + }, }; type FullBackend = TFullBackend; @@ -858,7 +861,7 @@ fn start_consensus_orchestrator( overseer_handle: OverseerHandle, announce_block: Arc>) + Send + Sync>, proposer_factory: ParachainProposerFactory, - end_lookahead_receiver: Option> + end_lookahead_receiver: Option>, ) { let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client) .expect("start_consensus_orchestrator: slot duration should exist"); From 30ca760e907f7b6de94d607cce4b006963c71b11 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 12 Mar 2024 19:44:11 -0700 Subject: [PATCH 17/35] toml fmt --- client/consensus/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/consensus/Cargo.toml b/client/consensus/Cargo.toml index 5121d63ae..f0677df20 100644 --- a/client/consensus/Cargo.toml +++ b/client/consensus/Cargo.toml @@ -64,8 +64,8 @@ async-trait = { workspace = true } futures = { workspace = true } log = { workspace = true } parity-scale-codec = { workspace = true, features = [ "derive" ] } -tracing = { workspace = true } tokio = { workspace = true } +tracing = { workspace = true } [dev-dependencies] cumulus-test-relay-sproof-builder = { workspace = true } From 4c719577a8e00842a1a7e3f32ff35e96f6f6cdf6 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Wed, 13 Mar 2024 08:00:52 -0700 Subject: [PATCH 18/35] change velocity to 1 --- container-chains/templates/frontier/runtime/src/lib.rs | 2 +- container-chains/templates/simple/runtime/src/lib.rs | 2 +- runtime/dancebox/src/lib.rs | 2 +- runtime/flashbox/src/lib.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/container-chains/templates/frontier/runtime/src/lib.rs b/container-chains/templates/frontier/runtime/src/lib.rs index 8991866b2..deb374435 100644 --- a/container-chains/templates/frontier/runtime/src/lib.rs +++ b/container-chains/templates/frontier/runtime/src/lib.rs @@ -512,7 +512,7 @@ parameter_types! { pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; -pub const BLOCK_PROCESSING_VELOCITY: u32 = 2; +pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< Runtime, diff --git a/container-chains/templates/simple/runtime/src/lib.rs b/container-chains/templates/simple/runtime/src/lib.rs index 5960c32ff..ffc375ba8 100644 --- a/container-chains/templates/simple/runtime/src/lib.rs +++ b/container-chains/templates/simple/runtime/src/lib.rs @@ -402,7 +402,7 @@ parameter_types! { pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; -pub const BLOCK_PROCESSING_VELOCITY: u32 = 2; +pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< Runtime, diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index 31bd940a0..a24d05252 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -480,7 +480,7 @@ parameter_types! { pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; -pub const BLOCK_PROCESSING_VELOCITY: u32 = 2; +pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< Runtime, diff --git a/runtime/flashbox/src/lib.rs b/runtime/flashbox/src/lib.rs index dad78c6a2..47336e02b 100644 --- a/runtime/flashbox/src/lib.rs +++ b/runtime/flashbox/src/lib.rs @@ -454,7 +454,7 @@ impl pallet_transaction_payment::Config for Runtime { } pub const RELAY_CHAIN_SLOT_DURATION_MILLIS: u32 = 6000; -pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 2; +pub const UNINCLUDED_SEGMENT_CAPACITY: u32 = 3; pub const BLOCK_PROCESSING_VELOCITY: u32 = 1; type ConsensusHook = pallet_async_backing::consensus_hook::FixedVelocityConsensusHook< From d868df197da88d9cf85ca194bcc8b75ad04346f7 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Wed, 13 Mar 2024 08:17:34 -0700 Subject: [PATCH 19/35] use proper polkadot-sdk branches --- Cargo.lock | 694 ++++++++++++++++++++++++++--------------------------- Cargo.toml | 352 +++++++++++++-------------- 2 files changed, 523 insertions(+), 523 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7260fdf7f..4c88b40d7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -433,7 +433,7 @@ checksum = "9b34d609dfbaf33d6889b2b7106d3ca345eacad44200913df5ba02bfd31d2ba9" [[package]] name = "asset-test-utils" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "assets-common", "cumulus-pallet-parachain-system", @@ -468,7 +468,7 @@ dependencies = [ [[package]] name = "assets-common" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -492,7 +492,7 @@ dependencies = [ [[package]] name = "async-backing-primitives" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "sp-api", "sp-consensus-slots", @@ -788,7 +788,7 @@ dependencies = [ [[package]] name = "binary-merkle-tree" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "hash-db", "log", @@ -1013,7 +1013,7 @@ dependencies = [ [[package]] name = "bp-header-chain" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-runtime", "finality-grandpa", @@ -1030,7 +1030,7 @@ dependencies = [ [[package]] name = "bp-messages" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-header-chain", "bp-runtime", @@ -1045,7 +1045,7 @@ dependencies = [ [[package]] name = "bp-parachains" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-header-chain", "bp-polkadot-core", @@ -1062,7 +1062,7 @@ dependencies = [ [[package]] name = "bp-polkadot-core" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-messages", "bp-runtime", @@ -1080,7 +1080,7 @@ dependencies = [ [[package]] name = "bp-relayers" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-messages", "bp-runtime", @@ -1094,7 +1094,7 @@ dependencies = [ [[package]] name = "bp-runtime" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -1117,7 +1117,7 @@ dependencies = [ [[package]] name = "bp-test-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-header-chain", "bp-parachains", @@ -1137,7 +1137,7 @@ dependencies = [ [[package]] name = "bp-xcm-bridge-hub" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "sp-std", ] @@ -1145,7 +1145,7 @@ dependencies = [ [[package]] name = "bp-xcm-bridge-hub-router" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -1156,7 +1156,7 @@ dependencies = [ [[package]] name = "bridge-runtime-common" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-header-chain", "bp-messages", @@ -1341,7 +1341,7 @@ dependencies = [ [[package]] name = "ccp-authorities-noting-inherent" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -1368,7 +1368,7 @@ dependencies = [ [[package]] name = "ccp-xcm" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "frame-support", "frame-system", @@ -2299,7 +2299,7 @@ dependencies = [ [[package]] name = "cumulus-client-cli" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "clap", "parity-scale-codec", @@ -2316,7 +2316,7 @@ dependencies = [ [[package]] name = "cumulus-client-collator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-client-consensus-common", "cumulus-client-network", @@ -2339,7 +2339,7 @@ dependencies = [ [[package]] name = "cumulus-client-consensus-aura" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-client-collator", @@ -2381,7 +2381,7 @@ dependencies = [ [[package]] name = "cumulus-client-consensus-common" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-client-pov-recovery", @@ -2410,7 +2410,7 @@ dependencies = [ [[package]] name = "cumulus-client-consensus-proposer" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "anyhow", "async-trait", @@ -2425,7 +2425,7 @@ dependencies = [ [[package]] name = "cumulus-client-network" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-relay-chain-interface", @@ -2448,7 +2448,7 @@ dependencies = [ [[package]] name = "cumulus-client-parachain-inherent" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2472,7 +2472,7 @@ dependencies = [ [[package]] name = "cumulus-client-pov-recovery" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2496,7 +2496,7 @@ dependencies = [ [[package]] name = "cumulus-client-service" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-client-cli", "cumulus-client-collator", @@ -2532,7 +2532,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-dmp-queue" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "frame-benchmarking", @@ -2550,7 +2550,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-parachain-system" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bytes", "cumulus-pallet-parachain-system-proc-macro", @@ -2584,7 +2584,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-parachain-system-proc-macro" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -2595,7 +2595,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-session-benchmarking" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -2609,7 +2609,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-xcm" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -2625,7 +2625,7 @@ dependencies = [ [[package]] name = "cumulus-pallet-xcmp-queue" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bounded-collections", "bp-xcm-bridge-hub-router", @@ -2650,7 +2650,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-aura" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "polkadot-core-primitives", @@ -2664,7 +2664,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-core" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "polkadot-core-primitives", @@ -2681,7 +2681,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-parachain-inherent" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2696,7 +2696,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-proof-size-hostfunction" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "sp-externalities", "sp-runtime-interface", @@ -2706,7 +2706,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-timestamp" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "futures 0.3.30", @@ -2719,7 +2719,7 @@ dependencies = [ [[package]] name = "cumulus-primitives-utility" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -2739,7 +2739,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-inprocess-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2763,7 +2763,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2781,7 +2781,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-minimal-node" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "async-trait", @@ -2822,7 +2822,7 @@ dependencies = [ [[package]] name = "cumulus-relay-chain-rpc-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -2861,7 +2861,7 @@ dependencies = [ [[package]] name = "cumulus-test-relay-sproof-builder" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "parity-scale-codec", @@ -3130,7 +3130,7 @@ dependencies = [ [[package]] name = "dc-orchestrator-chain-interface" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "async-trait", "cumulus-primitives-core", @@ -3351,7 +3351,7 @@ checksum = "9ea835d29036a4087793836fa931b08837ad5e957da9e23886b29586fb9b6650" [[package]] name = "dp-chain-state-snapshot" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "cumulus-primitives-core", "parity-scale-codec", @@ -3363,7 +3363,7 @@ dependencies = [ [[package]] name = "dp-collator-assignment" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -3383,7 +3383,7 @@ dependencies = [ [[package]] name = "dp-consensus" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -3400,7 +3400,7 @@ dependencies = [ [[package]] name = "dp-core" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -3415,7 +3415,7 @@ dependencies = [ [[package]] name = "dp-impl-tanssi-pallets-config" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "dp-consensus", "frame-support", @@ -3559,7 +3559,7 @@ dependencies = [ [[package]] name = "emulated-integration-tests-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "asset-test-utils", "bp-messages", @@ -3936,7 +3936,7 @@ dependencies = [ [[package]] name = "fc-api" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "async-trait", "fp-storage", @@ -3948,7 +3948,7 @@ dependencies = [ [[package]] name = "fc-cli" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "clap", "ethereum-types", @@ -3966,7 +3966,7 @@ dependencies = [ [[package]] name = "fc-consensus" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "async-trait", "fp-consensus", @@ -3982,7 +3982,7 @@ dependencies = [ [[package]] name = "fc-db" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "async-trait", "ethereum", @@ -4013,7 +4013,7 @@ dependencies = [ [[package]] name = "fc-mapping-sync" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "fc-db", "fc-storage", @@ -4036,7 +4036,7 @@ dependencies = [ [[package]] name = "fc-rpc" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "ethereum-types", @@ -4091,7 +4091,7 @@ dependencies = [ [[package]] name = "fc-rpc-core" version = "1.1.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "ethereum-types", @@ -4104,7 +4104,7 @@ dependencies = [ [[package]] name = "fc-storage" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "ethereum-types", @@ -4360,7 +4360,7 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "fork-tree" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", ] @@ -4377,7 +4377,7 @@ dependencies = [ [[package]] name = "fp-account" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "hex", "impl-serde", @@ -4396,7 +4396,7 @@ dependencies = [ [[package]] name = "fp-consensus" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "parity-scale-codec", @@ -4408,7 +4408,7 @@ dependencies = [ [[package]] name = "fp-ethereum" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "ethereum-types", @@ -4421,7 +4421,7 @@ dependencies = [ [[package]] name = "fp-evm" version = "3.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "evm", "frame-support", @@ -4437,7 +4437,7 @@ dependencies = [ [[package]] name = "fp-rpc" version = "3.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "ethereum-types", @@ -4454,7 +4454,7 @@ dependencies = [ [[package]] name = "fp-self-contained" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "frame-support", "parity-scale-codec", @@ -4466,7 +4466,7 @@ dependencies = [ [[package]] name = "fp-storage" version = "2.0.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "parity-scale-codec", "serde", @@ -4481,7 +4481,7 @@ checksum = "6c2141d6d6c8512188a7891b4b01590a45f6dac67afb4f255c4124dbb86d4eaa" [[package]] name = "frame-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-support-procedural", @@ -4506,7 +4506,7 @@ dependencies = [ [[package]] name = "frame-benchmarking-cli" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "Inflector", "array-bytes 6.2.2", @@ -4554,7 +4554,7 @@ dependencies = [ [[package]] name = "frame-election-provider-solution-type" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -4565,7 +4565,7 @@ dependencies = [ [[package]] name = "frame-election-provider-support" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-election-provider-solution-type", "frame-support", @@ -4582,7 +4582,7 @@ dependencies = [ [[package]] name = "frame-executive" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -4612,7 +4612,7 @@ dependencies = [ [[package]] name = "frame-remote-externalities" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "indicatif", @@ -4633,7 +4633,7 @@ dependencies = [ [[package]] name = "frame-support" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "aquamarine", "array-bytes 6.2.2", @@ -4674,7 +4674,7 @@ dependencies = [ [[package]] name = "frame-support-procedural" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "Inflector", "cfg-expr", @@ -4693,7 +4693,7 @@ dependencies = [ [[package]] name = "frame-support-procedural-tools" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support-procedural-tools-derive", "proc-macro-crate 3.1.0", @@ -4705,7 +4705,7 @@ dependencies = [ [[package]] name = "frame-support-procedural-tools-derive" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro2", "quote", @@ -4715,7 +4715,7 @@ dependencies = [ [[package]] name = "frame-system" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cfg-if", "docify", @@ -4735,7 +4735,7 @@ dependencies = [ [[package]] name = "frame-system-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -4750,7 +4750,7 @@ dependencies = [ [[package]] name = "frame-system-rpc-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "sp-api", @@ -4759,7 +4759,7 @@ dependencies = [ [[package]] name = "frame-try-runtime" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "parity-scale-codec", @@ -6778,7 +6778,7 @@ dependencies = [ [[package]] name = "mmr-gadget" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "log", @@ -6797,7 +6797,7 @@ dependencies = [ [[package]] name = "mmr-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "anyhow", "jsonrpsee", @@ -7118,7 +7118,7 @@ dependencies = [ [[package]] name = "nimbus-consensus" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "async-backing-primitives", "async-trait", @@ -7158,7 +7158,7 @@ dependencies = [ [[package]] name = "nimbus-primitives" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "async-trait", "frame-benchmarking", @@ -7574,7 +7574,7 @@ dependencies = [ [[package]] name = "pallet-asset-conversion" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7592,7 +7592,7 @@ dependencies = [ [[package]] name = "pallet-asset-rate" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7607,7 +7607,7 @@ dependencies = [ [[package]] name = "pallet-asset-tx-payment" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7625,7 +7625,7 @@ dependencies = [ [[package]] name = "pallet-assets" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7641,7 +7641,7 @@ dependencies = [ [[package]] name = "pallet-async-backing" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-primitives-core", @@ -7661,7 +7661,7 @@ dependencies = [ [[package]] name = "pallet-author-inherent" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "frame-benchmarking", "frame-support", @@ -7741,7 +7741,7 @@ dependencies = [ [[package]] name = "pallet-authority-discovery" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -7771,7 +7771,7 @@ dependencies = [ [[package]] name = "pallet-authorship" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -7785,7 +7785,7 @@ dependencies = [ [[package]] name = "pallet-babe" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7809,7 +7809,7 @@ dependencies = [ [[package]] name = "pallet-bags-list" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "aquamarine", "docify", @@ -7831,7 +7831,7 @@ dependencies = [ [[package]] name = "pallet-balances" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7846,7 +7846,7 @@ dependencies = [ [[package]] name = "pallet-base-fee" version = "1.0.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "fp-evm", "frame-support", @@ -7860,7 +7860,7 @@ dependencies = [ [[package]] name = "pallet-beefy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -7880,7 +7880,7 @@ dependencies = [ [[package]] name = "pallet-beefy-mmr" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "binary-merkle-tree", @@ -7905,7 +7905,7 @@ dependencies = [ [[package]] name = "pallet-bounties" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -7923,7 +7923,7 @@ dependencies = [ [[package]] name = "pallet-bridge-grandpa" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-header-chain", "bp-runtime", @@ -7944,7 +7944,7 @@ dependencies = [ [[package]] name = "pallet-bridge-messages" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-messages", "bp-runtime", @@ -7962,7 +7962,7 @@ dependencies = [ [[package]] name = "pallet-bridge-parachains" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-header-chain", "bp-parachains", @@ -7983,7 +7983,7 @@ dependencies = [ [[package]] name = "pallet-bridge-relayers" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-messages", "bp-relayers", @@ -8003,7 +8003,7 @@ dependencies = [ [[package]] name = "pallet-broker" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "frame-benchmarking", @@ -8020,7 +8020,7 @@ dependencies = [ [[package]] name = "pallet-cc-authorities-noting" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "ccp-authorities-noting-inherent", "cumulus-pallet-parachain-system", @@ -8049,7 +8049,7 @@ dependencies = [ [[package]] name = "pallet-child-bounties" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8100,7 +8100,7 @@ dependencies = [ [[package]] name = "pallet-collator-selection" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8119,7 +8119,7 @@ dependencies = [ [[package]] name = "pallet-collective" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8154,7 +8154,7 @@ dependencies = [ [[package]] name = "pallet-conviction-voting" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "assert_matches", "frame-benchmarking", @@ -8195,7 +8195,7 @@ dependencies = [ [[package]] name = "pallet-democracy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8213,7 +8213,7 @@ dependencies = [ [[package]] name = "pallet-election-provider-multi-phase" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8236,7 +8236,7 @@ dependencies = [ [[package]] name = "pallet-election-provider-support-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8250,7 +8250,7 @@ dependencies = [ [[package]] name = "pallet-elections-phragmen" version = "5.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8269,7 +8269,7 @@ dependencies = [ [[package]] name = "pallet-ethereum" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "ethereum", "ethereum-types", @@ -8292,7 +8292,7 @@ dependencies = [ [[package]] name = "pallet-evm" version = "6.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "environmental", "evm", @@ -8318,7 +8318,7 @@ dependencies = [ [[package]] name = "pallet-evm-chain-id" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "frame-support", "frame-system", @@ -8329,7 +8329,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-balances-erc20" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "fp-evm", "frame-support", @@ -8352,7 +8352,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-batch" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "evm", "fp-evm", @@ -8373,7 +8373,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-call-permit" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "evm", "fp-evm", @@ -8396,7 +8396,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-modexp" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "fp-evm", "num", @@ -8405,7 +8405,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-sha3fips" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "fp-evm", "tiny-keccak", @@ -8414,7 +8414,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-simple" version = "2.0.0-dev" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "fp-evm", "ripemd", @@ -8424,7 +8424,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-xcm-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "fp-evm", "frame-support", @@ -8446,7 +8446,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompileset-assets-erc20" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "fp-evm", "frame-support", @@ -8467,7 +8467,7 @@ dependencies = [ [[package]] name = "pallet-fast-unstake" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "frame-benchmarking", @@ -8486,7 +8486,7 @@ dependencies = [ [[package]] name = "pallet-foreign-asset-creator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "frame-benchmarking", "frame-support", @@ -8505,7 +8505,7 @@ dependencies = [ [[package]] name = "pallet-grandpa" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8528,7 +8528,7 @@ dependencies = [ [[package]] name = "pallet-hotfix-sufficients" version = "1.0.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "frame-benchmarking", "frame-support", @@ -8544,7 +8544,7 @@ dependencies = [ [[package]] name = "pallet-identity" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "enumflags2", "frame-benchmarking", @@ -8561,7 +8561,7 @@ dependencies = [ [[package]] name = "pallet-im-online" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8581,7 +8581,7 @@ dependencies = [ [[package]] name = "pallet-indices" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8658,7 +8658,7 @@ dependencies = [ [[package]] name = "pallet-maintenance-mode" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -8674,7 +8674,7 @@ dependencies = [ [[package]] name = "pallet-membership" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8691,7 +8691,7 @@ dependencies = [ [[package]] name = "pallet-message-queue" version = "7.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "environmental", "frame-benchmarking", @@ -8711,7 +8711,7 @@ dependencies = [ [[package]] name = "pallet-migrations" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "frame-benchmarking", "frame-support", @@ -8730,7 +8730,7 @@ dependencies = [ [[package]] name = "pallet-mmr" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8748,7 +8748,7 @@ dependencies = [ [[package]] name = "pallet-multisig" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8764,7 +8764,7 @@ dependencies = [ [[package]] name = "pallet-nis" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8780,7 +8780,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -8799,7 +8799,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools-benchmarking" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8819,7 +8819,7 @@ dependencies = [ [[package]] name = "pallet-nomination-pools-runtime-api" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "pallet-nomination-pools", "parity-scale-codec", @@ -8830,7 +8830,7 @@ dependencies = [ [[package]] name = "pallet-offences" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -8847,7 +8847,7 @@ dependencies = [ [[package]] name = "pallet-offences-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -8895,7 +8895,7 @@ dependencies = [ [[package]] name = "pallet-preimage" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8912,7 +8912,7 @@ dependencies = [ [[package]] name = "pallet-proxy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8927,7 +8927,7 @@ dependencies = [ [[package]] name = "pallet-ranked-collective" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8945,7 +8945,7 @@ dependencies = [ [[package]] name = "pallet-recovery" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -8960,7 +8960,7 @@ dependencies = [ [[package]] name = "pallet-referenda" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "assert_matches", "frame-benchmarking", @@ -9012,7 +9012,7 @@ dependencies = [ [[package]] name = "pallet-relay-storage-roots" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-primitives-core", @@ -9035,7 +9035,7 @@ dependencies = [ [[package]] name = "pallet-root-testing" version = "1.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -9050,7 +9050,7 @@ dependencies = [ [[package]] name = "pallet-scheduler" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "frame-benchmarking", @@ -9088,7 +9088,7 @@ dependencies = [ [[package]] name = "pallet-session" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -9110,7 +9110,7 @@ dependencies = [ [[package]] name = "pallet-session-benchmarking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9127,7 +9127,7 @@ dependencies = [ [[package]] name = "pallet-society" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9145,7 +9145,7 @@ dependencies = [ [[package]] name = "pallet-staking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-election-provider-support", @@ -9168,7 +9168,7 @@ dependencies = [ [[package]] name = "pallet-staking-reward-curve" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -9179,7 +9179,7 @@ dependencies = [ [[package]] name = "pallet-staking-reward-fn" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "log", "sp-arithmetic", @@ -9188,7 +9188,7 @@ dependencies = [ [[package]] name = "pallet-staking-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "sp-api", @@ -9198,7 +9198,7 @@ dependencies = [ [[package]] name = "pallet-state-trie-migration" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9239,7 +9239,7 @@ dependencies = [ [[package]] name = "pallet-sudo" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "frame-benchmarking", @@ -9255,7 +9255,7 @@ dependencies = [ [[package]] name = "pallet-timestamp" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "frame-benchmarking", @@ -9275,7 +9275,7 @@ dependencies = [ [[package]] name = "pallet-tips" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9294,7 +9294,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -9310,7 +9310,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "jsonrpsee", "pallet-transaction-payment-rpc-runtime-api", @@ -9326,7 +9326,7 @@ dependencies = [ [[package]] name = "pallet-transaction-payment-rpc-runtime-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "pallet-transaction-payment", "parity-scale-codec", @@ -9338,7 +9338,7 @@ dependencies = [ [[package]] name = "pallet-treasury" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "frame-benchmarking", @@ -9357,7 +9357,7 @@ dependencies = [ [[package]] name = "pallet-tx-pause" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "frame-benchmarking", @@ -9375,7 +9375,7 @@ dependencies = [ [[package]] name = "pallet-utility" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9391,7 +9391,7 @@ dependencies = [ [[package]] name = "pallet-vesting" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9406,7 +9406,7 @@ dependencies = [ [[package]] name = "pallet-whitelist" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9421,7 +9421,7 @@ dependencies = [ [[package]] name = "pallet-xcm" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bounded-collections", "frame-benchmarking", @@ -9444,7 +9444,7 @@ dependencies = [ [[package]] name = "pallet-xcm-benchmarks" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-benchmarking", "frame-support", @@ -9463,7 +9463,7 @@ dependencies = [ [[package]] name = "pallet-xcm-bridge-hub-router" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bp-xcm-bridge-hub-router", "frame-benchmarking", @@ -9482,7 +9482,7 @@ dependencies = [ [[package]] name = "pallet-xcm-executor-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "frame-benchmarking", "frame-support", @@ -9500,7 +9500,7 @@ dependencies = [ [[package]] name = "parachains-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "cumulus-primitives-utility", @@ -9537,7 +9537,7 @@ dependencies = [ [[package]] name = "parachains-runtimes-test-utils" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "assets-common", "cumulus-pallet-parachain-system", @@ -9890,7 +9890,7 @@ checksum = "626dec3cac7cc0e1577a2ec3fc496277ec2baa084bebad95bb6fdbfae235f84c" [[package]] name = "polkadot-approval-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "futures 0.3.30", @@ -9910,7 +9910,7 @@ dependencies = [ [[package]] name = "polkadot-availability-bitfield-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "always-assert", "futures 0.3.30", @@ -9926,7 +9926,7 @@ dependencies = [ [[package]] name = "polkadot-availability-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "derive_more", "fatality", @@ -9949,7 +9949,7 @@ dependencies = [ [[package]] name = "polkadot-availability-recovery" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "fatality", @@ -9972,7 +9972,7 @@ dependencies = [ [[package]] name = "polkadot-cli" version = "1.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cfg-if", "clap", @@ -10000,7 +10000,7 @@ dependencies = [ [[package]] name = "polkadot-collator-protocol" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "fatality", @@ -10022,7 +10022,7 @@ dependencies = [ [[package]] name = "polkadot-core-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -10034,7 +10034,7 @@ dependencies = [ [[package]] name = "polkadot-dispute-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "derive_more", "fatality", @@ -10059,7 +10059,7 @@ dependencies = [ [[package]] name = "polkadot-erasure-coding" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "polkadot-node-primitives", @@ -10073,7 +10073,7 @@ dependencies = [ [[package]] name = "polkadot-gossip-support" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "futures-timer", @@ -10094,7 +10094,7 @@ dependencies = [ [[package]] name = "polkadot-network-bridge" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "always-assert", "async-trait", @@ -10117,7 +10117,7 @@ dependencies = [ [[package]] name = "polkadot-node-collation-generation" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "parity-scale-codec", @@ -10135,7 +10135,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-approval-voting" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "derive_more", @@ -10168,7 +10168,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-av-store" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "futures 0.3.30", @@ -10190,7 +10190,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-backing" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "fatality", @@ -10209,7 +10209,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-bitfield-signing" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "polkadot-node-subsystem", @@ -10224,7 +10224,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-candidate-validation" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -10245,7 +10245,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-chain-api" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "polkadot-node-metrics", @@ -10259,7 +10259,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-chain-selection" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "futures-timer", @@ -10276,7 +10276,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-dispute-coordinator" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "fatality", "futures 0.3.30", @@ -10295,7 +10295,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-parachains-inherent" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -10312,7 +10312,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-prospective-parachains" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "fatality", @@ -10329,7 +10329,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-provisioner" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "fatality", @@ -10346,7 +10346,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "always-assert", "array-bytes 6.2.2", @@ -10379,7 +10379,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf-checker" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "polkadot-node-primitives", @@ -10395,7 +10395,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-pvf-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cfg-if", "cpu-time", @@ -10420,7 +10420,7 @@ dependencies = [ [[package]] name = "polkadot-node-core-runtime-api" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "polkadot-node-metrics", @@ -10435,7 +10435,7 @@ dependencies = [ [[package]] name = "polkadot-node-jaeger" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "lazy_static", "log", @@ -10453,7 +10453,7 @@ dependencies = [ [[package]] name = "polkadot-node-metrics" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bs58 0.5.0", "futures 0.3.30", @@ -10472,7 +10472,7 @@ dependencies = [ [[package]] name = "polkadot-node-network-protocol" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-channel 1.9.0", "async-trait", @@ -10496,7 +10496,7 @@ dependencies = [ [[package]] name = "polkadot-node-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "bounded-vec", @@ -10519,7 +10519,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "polkadot-node-jaeger", "polkadot-node-subsystem-types", @@ -10529,7 +10529,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-types" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "bitvec", @@ -10557,7 +10557,7 @@ dependencies = [ [[package]] name = "polkadot-node-subsystem-util" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "derive_more", @@ -10592,7 +10592,7 @@ dependencies = [ [[package]] name = "polkadot-overseer" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -10614,7 +10614,7 @@ dependencies = [ [[package]] name = "polkadot-parachain-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bounded-collections", "derive_more", @@ -10631,7 +10631,7 @@ dependencies = [ [[package]] name = "polkadot-primitives" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "hex-literal 0.4.1", @@ -10657,7 +10657,7 @@ dependencies = [ [[package]] name = "polkadot-rpc" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "jsonrpsee", "mmr-rpc", @@ -10690,7 +10690,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-common" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitvec", "frame-benchmarking", @@ -10742,7 +10742,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-metrics" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bs58 0.5.0", "frame-benchmarking", @@ -10755,7 +10755,7 @@ dependencies = [ [[package]] name = "polkadot-runtime-parachains" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bitflags 1.3.2", "bitvec", @@ -10804,7 +10804,7 @@ dependencies = [ [[package]] name = "polkadot-service" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "frame-benchmarking", @@ -10921,7 +10921,7 @@ dependencies = [ [[package]] name = "polkadot-statement-distribution" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "arrayvec 0.7.4", "bitvec", @@ -10944,7 +10944,7 @@ dependencies = [ [[package]] name = "polkadot-statement-table" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "polkadot-primitives", @@ -11025,7 +11025,7 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "precompile-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "environmental", "evm", @@ -11050,7 +11050,7 @@ dependencies = [ [[package]] name = "precompile-utils-macro" version = "0.1.0" -source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0-lookahead#80c1fd9d2817e9bff158f542263d21e746db77de" +source = "git+https://github.com/moondance-labs/frontier?branch=tanssi-polkadot-v1.6.0#4414529b910e8cf802969b11505a14665e4a55d1" dependencies = [ "case", "num_enum", @@ -11718,7 +11718,7 @@ dependencies = [ [[package]] name = "rococo-runtime" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "binary-merkle-tree", "frame-benchmarking", @@ -11814,7 +11814,7 @@ dependencies = [ [[package]] name = "rococo-runtime-constants" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "polkadot-primitives", @@ -12075,7 +12075,7 @@ dependencies = [ [[package]] name = "sc-allocator" version = "4.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "log", "sp-core", @@ -12086,7 +12086,7 @@ dependencies = [ [[package]] name = "sc-authority-discovery" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -12115,7 +12115,7 @@ dependencies = [ [[package]] name = "sc-basic-authorship" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "futures-timer", @@ -12137,7 +12137,7 @@ dependencies = [ [[package]] name = "sc-block-builder" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "sp-api", @@ -12152,7 +12152,7 @@ dependencies = [ [[package]] name = "sc-chain-spec" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "docify", @@ -12177,7 +12177,7 @@ dependencies = [ [[package]] name = "sc-chain-spec-derive" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -12188,7 +12188,7 @@ dependencies = [ [[package]] name = "sc-cli" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "bip39", @@ -12232,7 +12232,7 @@ dependencies = [ [[package]] name = "sc-client-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "fnv", "futures 0.3.30", @@ -12259,7 +12259,7 @@ dependencies = [ [[package]] name = "sc-client-db" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "hash-db", "kvdb", @@ -12285,7 +12285,7 @@ dependencies = [ [[package]] name = "sc-consensus" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -12310,7 +12310,7 @@ dependencies = [ [[package]] name = "sc-consensus-aura" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -12339,7 +12339,7 @@ dependencies = [ [[package]] name = "sc-consensus-babe" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "fork-tree", @@ -12374,7 +12374,7 @@ dependencies = [ [[package]] name = "sc-consensus-babe-rpc" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "jsonrpsee", @@ -12396,7 +12396,7 @@ dependencies = [ [[package]] name = "sc-consensus-beefy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12431,7 +12431,7 @@ dependencies = [ [[package]] name = "sc-consensus-beefy-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "jsonrpsee", @@ -12450,7 +12450,7 @@ dependencies = [ [[package]] name = "sc-consensus-epochs" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "fork-tree", "parity-scale-codec", @@ -12463,7 +12463,7 @@ dependencies = [ [[package]] name = "sc-consensus-grandpa" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ahash 0.8.8", "array-bytes 6.2.2", @@ -12505,7 +12505,7 @@ dependencies = [ [[package]] name = "sc-consensus-grandpa-rpc" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "finality-grandpa", "futures 0.3.30", @@ -12525,7 +12525,7 @@ dependencies = [ [[package]] name = "sc-consensus-manual-seal" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "assert_matches", "async-trait", @@ -12560,7 +12560,7 @@ dependencies = [ [[package]] name = "sc-consensus-slots" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -12583,7 +12583,7 @@ dependencies = [ [[package]] name = "sc-executor" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "log", "parity-scale-codec", @@ -12606,7 +12606,7 @@ dependencies = [ [[package]] name = "sc-executor-common" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "sc-allocator", @@ -12619,7 +12619,7 @@ dependencies = [ [[package]] name = "sc-executor-wasmtime" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "anyhow", "cfg-if", @@ -12639,7 +12639,7 @@ dependencies = [ [[package]] name = "sc-informant" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ansi_term", "futures 0.3.30", @@ -12656,7 +12656,7 @@ dependencies = [ [[package]] name = "sc-keystore" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "parking_lot 0.12.1", @@ -12670,7 +12670,7 @@ dependencies = [ [[package]] name = "sc-mixnet" version = "0.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 4.2.0", "arrayvec 0.7.4", @@ -12699,7 +12699,7 @@ dependencies = [ [[package]] name = "sc-network" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12742,7 +12742,7 @@ dependencies = [ [[package]] name = "sc-network-bitswap" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-channel 1.9.0", "cid", @@ -12762,7 +12762,7 @@ dependencies = [ [[package]] name = "sc-network-common" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "bitflags 1.3.2", @@ -12779,7 +12779,7 @@ dependencies = [ [[package]] name = "sc-network-gossip" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ahash 0.8.8", "futures 0.3.30", @@ -12798,7 +12798,7 @@ dependencies = [ [[package]] name = "sc-network-light" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12819,7 +12819,7 @@ dependencies = [ [[package]] name = "sc-network-sync" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "async-channel 1.9.0", @@ -12855,7 +12855,7 @@ dependencies = [ [[package]] name = "sc-network-test" version = "0.8.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -12886,7 +12886,7 @@ dependencies = [ [[package]] name = "sc-network-transactions" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "futures 0.3.30", @@ -12905,7 +12905,7 @@ dependencies = [ [[package]] name = "sc-offchain" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "bytes", @@ -12939,7 +12939,7 @@ dependencies = [ [[package]] name = "sc-proposer-metrics" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "log", "substrate-prometheus-endpoint", @@ -12948,7 +12948,7 @@ dependencies = [ [[package]] name = "sc-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "jsonrpsee", @@ -12980,7 +12980,7 @@ dependencies = [ [[package]] name = "sc-rpc-api" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -13000,7 +13000,7 @@ dependencies = [ [[package]] name = "sc-rpc-server" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "http", "jsonrpsee", @@ -13015,7 +13015,7 @@ dependencies = [ [[package]] name = "sc-rpc-spec-v2" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "futures 0.3.30", @@ -13044,7 +13044,7 @@ dependencies = [ [[package]] name = "sc-service" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "directories", @@ -13107,7 +13107,7 @@ dependencies = [ [[package]] name = "sc-state-db" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "log", "parity-scale-codec", @@ -13118,7 +13118,7 @@ dependencies = [ [[package]] name = "sc-storage-monitor" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "clap", "fs4", @@ -13131,7 +13131,7 @@ dependencies = [ [[package]] name = "sc-sync-state-rpc" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -13150,7 +13150,7 @@ dependencies = [ [[package]] name = "sc-sysinfo" version = "6.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "derive_more", "futures 0.3.30", @@ -13170,7 +13170,7 @@ dependencies = [ [[package]] name = "sc-telemetry" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "chrono", "futures 0.3.30", @@ -13189,7 +13189,7 @@ dependencies = [ [[package]] name = "sc-tracing" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ansi_term", "chrono", @@ -13219,7 +13219,7 @@ dependencies = [ [[package]] name = "sc-tracing-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", @@ -13230,7 +13230,7 @@ dependencies = [ [[package]] name = "sc-transaction-pool" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -13256,7 +13256,7 @@ dependencies = [ [[package]] name = "sc-transaction-pool-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -13272,7 +13272,7 @@ dependencies = [ [[package]] name = "sc-utils" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-channel 1.9.0", "futures 0.3.30", @@ -13727,7 +13727,7 @@ dependencies = [ [[package]] name = "slot-range-helper" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "enumn", "parity-scale-codec", @@ -13921,7 +13921,7 @@ dependencies = [ [[package]] name = "sp-api" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "hash-db", "log", @@ -13942,7 +13942,7 @@ dependencies = [ [[package]] name = "sp-api-proc-macro" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "Inflector", "blake2 0.10.6", @@ -13956,7 +13956,7 @@ dependencies = [ [[package]] name = "sp-application-crypto" version = "23.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -13969,7 +13969,7 @@ dependencies = [ [[package]] name = "sp-arithmetic" version = "16.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "integer-sqrt", "num-traits", @@ -13983,7 +13983,7 @@ dependencies = [ [[package]] name = "sp-authority-discovery" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -13996,7 +13996,7 @@ dependencies = [ [[package]] name = "sp-block-builder" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "sp-api", "sp-inherents", @@ -14007,7 +14007,7 @@ dependencies = [ [[package]] name = "sp-blockchain" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "log", @@ -14025,7 +14025,7 @@ dependencies = [ [[package]] name = "sp-consensus" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "futures 0.3.30", @@ -14040,7 +14040,7 @@ dependencies = [ [[package]] name = "sp-consensus-aura" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "parity-scale-codec", @@ -14057,7 +14057,7 @@ dependencies = [ [[package]] name = "sp-consensus-babe" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "parity-scale-codec", @@ -14076,7 +14076,7 @@ dependencies = [ [[package]] name = "sp-consensus-beefy" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "lazy_static", "parity-scale-codec", @@ -14095,7 +14095,7 @@ dependencies = [ [[package]] name = "sp-consensus-grandpa" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "finality-grandpa", "log", @@ -14113,7 +14113,7 @@ dependencies = [ [[package]] name = "sp-consensus-slots" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -14125,7 +14125,7 @@ dependencies = [ [[package]] name = "sp-core" version = "21.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "bip39", @@ -14170,7 +14170,7 @@ dependencies = [ [[package]] name = "sp-core-hashing" version = "9.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "blake2b_simd", "byteorder", @@ -14183,7 +14183,7 @@ dependencies = [ [[package]] name = "sp-core-hashing-proc-macro" version = "9.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "quote", "sp-core-hashing", @@ -14193,7 +14193,7 @@ dependencies = [ [[package]] name = "sp-database" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "kvdb", "parking_lot 0.12.1", @@ -14202,7 +14202,7 @@ dependencies = [ [[package]] name = "sp-debug-derive" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "proc-macro2", "quote", @@ -14212,7 +14212,7 @@ dependencies = [ [[package]] name = "sp-externalities" version = "0.19.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "environmental", "parity-scale-codec", @@ -14223,7 +14223,7 @@ dependencies = [ [[package]] name = "sp-genesis-builder" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "serde_json", "sp-api", @@ -14234,7 +14234,7 @@ dependencies = [ [[package]] name = "sp-inherents" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "impl-trait-for-tuples", @@ -14248,7 +14248,7 @@ dependencies = [ [[package]] name = "sp-io" version = "23.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bytes", "ed25519-dalek", @@ -14272,7 +14272,7 @@ dependencies = [ [[package]] name = "sp-keyring" version = "24.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "sp-core", "sp-runtime", @@ -14282,7 +14282,7 @@ dependencies = [ [[package]] name = "sp-keystore" version = "0.27.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "parking_lot 0.12.1", @@ -14294,7 +14294,7 @@ dependencies = [ [[package]] name = "sp-maybe-compressed-blob" version = "4.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "thiserror", "zstd 0.12.4", @@ -14303,7 +14303,7 @@ dependencies = [ [[package]] name = "sp-metadata-ir" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-metadata", "parity-scale-codec", @@ -14314,7 +14314,7 @@ dependencies = [ [[package]] name = "sp-mixnet" version = "0.1.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -14326,7 +14326,7 @@ dependencies = [ [[package]] name = "sp-mmr-primitives" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ckb-merkle-mountain-range", "log", @@ -14344,7 +14344,7 @@ dependencies = [ [[package]] name = "sp-npos-elections" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -14358,7 +14358,7 @@ dependencies = [ [[package]] name = "sp-offchain" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "sp-api", "sp-core", @@ -14368,7 +14368,7 @@ dependencies = [ [[package]] name = "sp-panic-handler" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "backtrace", "lazy_static", @@ -14378,7 +14378,7 @@ dependencies = [ [[package]] name = "sp-rpc" version = "6.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "rustc-hash", "serde", @@ -14388,7 +14388,7 @@ dependencies = [ [[package]] name = "sp-runtime" version = "24.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "docify", "either", @@ -14412,7 +14412,7 @@ dependencies = [ [[package]] name = "sp-runtime-interface" version = "17.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bytes", "impl-trait-for-tuples", @@ -14430,7 +14430,7 @@ dependencies = [ [[package]] name = "sp-runtime-interface-proc-macro" version = "11.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "Inflector", "expander 2.0.0", @@ -14443,7 +14443,7 @@ dependencies = [ [[package]] name = "sp-session" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "scale-info", @@ -14458,7 +14458,7 @@ dependencies = [ [[package]] name = "sp-staking" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "impl-trait-for-tuples", "parity-scale-codec", @@ -14472,7 +14472,7 @@ dependencies = [ [[package]] name = "sp-state-machine" version = "0.28.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "hash-db", "log", @@ -14493,7 +14493,7 @@ dependencies = [ [[package]] name = "sp-statement-store" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "aes-gcm", "curve25519-dalek 4.1.2", @@ -14517,12 +14517,12 @@ dependencies = [ [[package]] name = "sp-std" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" [[package]] name = "sp-storage" version = "13.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "impl-serde", "parity-scale-codec", @@ -14535,7 +14535,7 @@ dependencies = [ [[package]] name = "sp-timestamp" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "parity-scale-codec", @@ -14548,7 +14548,7 @@ dependencies = [ [[package]] name = "sp-tracing" version = "10.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "sp-std", @@ -14560,7 +14560,7 @@ dependencies = [ [[package]] name = "sp-transaction-pool" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "sp-api", "sp-runtime", @@ -14569,7 +14569,7 @@ dependencies = [ [[package]] name = "sp-transaction-storage-proof" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "parity-scale-codec", @@ -14584,7 +14584,7 @@ dependencies = [ [[package]] name = "sp-trie" version = "22.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ahash 0.8.8", "hash-db", @@ -14608,7 +14608,7 @@ dependencies = [ [[package]] name = "sp-version" version = "22.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "impl-serde", "parity-scale-codec", @@ -14625,7 +14625,7 @@ dependencies = [ [[package]] name = "sp-version-proc-macro" version = "8.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "parity-scale-codec", "proc-macro2", @@ -14636,7 +14636,7 @@ dependencies = [ [[package]] name = "sp-wasm-interface" version = "14.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "anyhow", "impl-trait-for-tuples", @@ -14649,7 +14649,7 @@ dependencies = [ [[package]] name = "sp-weights" version = "20.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "bounded-collections", "parity-scale-codec", @@ -14844,7 +14844,7 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "staging-parachain-info" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -14858,7 +14858,7 @@ dependencies = [ [[package]] name = "staging-xcm" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "bounded-collections", @@ -14876,7 +14876,7 @@ dependencies = [ [[package]] name = "staging-xcm-builder" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "frame-system", @@ -14898,7 +14898,7 @@ dependencies = [ [[package]] name = "staging-xcm-executor" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "environmental", "frame-benchmarking", @@ -15026,12 +15026,12 @@ dependencies = [ [[package]] name = "substrate-build-script-utils" version = "3.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" [[package]] name = "substrate-frame-rpc-system" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-system-rpc-runtime-api", "futures 0.3.30", @@ -15050,7 +15050,7 @@ dependencies = [ [[package]] name = "substrate-prometheus-endpoint" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "hyper", "log", @@ -15062,7 +15062,7 @@ dependencies = [ [[package]] name = "substrate-rpc-client" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "jsonrpsee", @@ -15075,7 +15075,7 @@ dependencies = [ [[package]] name = "substrate-state-trie-migration-rpc" version = "4.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "jsonrpsee", "parity-scale-codec", @@ -15092,7 +15092,7 @@ dependencies = [ [[package]] name = "substrate-test-client" version = "2.0.1" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "async-trait", @@ -15118,7 +15118,7 @@ dependencies = [ [[package]] name = "substrate-test-runtime" version = "2.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "array-bytes 6.2.2", "frame-executive", @@ -15159,7 +15159,7 @@ dependencies = [ [[package]] name = "substrate-test-runtime-client" version = "2.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "futures 0.3.30", "sc-block-builder", @@ -15177,7 +15177,7 @@ dependencies = [ [[package]] name = "substrate-wasm-builder" version = "5.0.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "ansi_term", "build-helper", @@ -15465,7 +15465,7 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "test-relay-sproof-builder" version = "0.1.0" -source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0-lookahead#43a207fa1036fcc06772542e1e26dea57aab0281" +source = "git+https://github.com/moondance-labs/dancekit?branch=tanssi-polkadot-v1.6.0#ae7ebc2a029c51e28d3d62c95723cfd21176cee2" dependencies = [ "cumulus-primitives-core", "dp-collator-assignment", @@ -15945,7 +15945,7 @@ dependencies = [ [[package]] name = "tracing-gum" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "coarsetime", "polkadot-primitives", @@ -15956,7 +15956,7 @@ dependencies = [ [[package]] name = "tracing-gum-proc-macro" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "expander 2.0.0", "proc-macro-crate 3.1.0", @@ -16086,7 +16086,7 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "try-runtime-cli" version = "0.10.0-dev" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "async-trait", "clap", @@ -16770,7 +16770,7 @@ checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" [[package]] name = "westend-runtime" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "binary-merkle-tree", "bitvec", @@ -16876,7 +16876,7 @@ dependencies = [ [[package]] name = "westend-runtime-constants" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "frame-support", "polkadot-primitives", @@ -17255,7 +17255,7 @@ dependencies = [ [[package]] name = "xcm-emulator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-pallet-xcmp-queue", @@ -17290,7 +17290,7 @@ dependencies = [ [[package]] name = "xcm-primitives" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0-lookahead#089dbb7ac8f24e310011a1e2e0ca5347353157fd" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" dependencies = [ "sp-runtime", ] @@ -17298,7 +17298,7 @@ dependencies = [ [[package]] name = "xcm-procedural" version = "1.0.0" -source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0-lookahead#2bd7c57881eb6204ebe7b08c1001bfab010138c0" +source = "git+https://github.com/moondance-labs/polkadot-sdk?branch=tanssi-polkadot-v1.6.0#49f36c06480dc6db9d8a4f06e0c4bcc4d8c4d18d" dependencies = [ "Inflector", "proc-macro2", diff --git a/Cargo.toml b/Cargo.toml index 982ce8778..be7754549 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -65,207 +65,207 @@ tp-maths = { path = "primitives/maths", default-features = false } tp-traits = { path = "primitives/traits", default-features = false } # Dancekit (wasm) -ccp-authorities-noting-inherent = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -ccp-xcm = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -dp-chain-state-snapshot = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -dp-collator-assignment = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -dp-consensus = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -dp-impl-tanssi-pallets-config = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +ccp-authorities-noting-inherent = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +ccp-xcm = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +dp-chain-state-snapshot = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +dp-collator-assignment = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +dp-consensus = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +dp-impl-tanssi-pallets-config = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } -dp-core = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-cc-authorities-noting = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-xcm-executor-utils = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -test-relay-sproof-builder = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +dp-core = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-cc-authorities-noting = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-xcm-executor-utils = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +test-relay-sproof-builder = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Dancekit (client) -dc-orchestrator-chain-interface = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0-lookahead" } +dc-orchestrator-chain-interface = { git = "https://github.com/moondance-labs/dancekit", branch = "tanssi-polkadot-v1.6.0" } # Moonkit (wasm) -async-backing-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -nimbus-consensus = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead" } -nimbus-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-async-backing = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-author-inherent = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-balances-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-batch = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-call-permit = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-xcm-utils = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompileset-assets-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-foreign-asset-creator = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-maintenance-mode = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-migrations = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-relay-storage-roots = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -xcm-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +async-backing-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +nimbus-consensus = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0" } +nimbus-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-async-backing = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-author-inherent = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-balances-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-batch = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-call-permit = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-xcm-utils = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompileset-assets-erc20 = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-foreign-asset-creator = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-maintenance-mode = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-migrations = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-relay-storage-roots = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } +xcm-primitives = { git = "https://github.com/moondance-labs/moonkit", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Substrate (wasm) -frame-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -frame-executive = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -frame-support = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "4.0.0-dev", default-features = false } -frame-system = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "4.0.0-dev", default-features = false } -frame-system-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -frame-system-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -frame-try-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-asset-rate = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-assets = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-balances = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-identity = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-message-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-multisig = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-proxy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-root-testing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-sudo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-transaction-payment = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-treasury = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-tx-pause = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +frame-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +frame-executive = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +frame-support = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "4.0.0-dev", default-features = false } +frame-system = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "4.0.0-dev", default-features = false } +frame-system-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +frame-system-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +frame-try-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-asset-rate = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-assets = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-balances = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-identity = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-message-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-multisig = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-proxy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-root-testing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-sudo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-transaction-payment = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-treasury = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-tx-pause = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } parity-scale-codec = { version = "3.0.0", default-features = false, features = [ "derive", "max-encoded-len" ] } scale-info = { version = "2.10.0", default-features = false } -sp-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-application-crypto = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-consensus-babe = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-consensus-beefy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-core = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "21.0.0", default-features = false } -sp-debug-derive = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-inherents = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-io = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "23.0.0", default-features = false } -sp-keyring = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "24.0.0", default-features = false } -sp-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0-lookahead", version = "24.0.0", default-features = false } -sp-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-state-machine = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-std = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-trie = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-version = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +sp-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-application-crypto = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-consensus-babe = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-consensus-beefy = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-core = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "21.0.0", default-features = false } +sp-debug-derive = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-inherents = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-io = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "23.0.0", default-features = false } +sp-keyring = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "24.0.0", default-features = false } +sp-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk.git", branch = "tanssi-polkadot-v1.6.0", version = "24.0.0", default-features = false } +sp-session = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-state-machine = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-std = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-trie = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-version = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Substrate (client) -frame-benchmarking-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -pallet-transaction-payment-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-transaction-payment-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sc-basic-authorship = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-chain-spec = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-client-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-consensus-grandpa = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-consensus-manual-seal = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-network-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-network-sync = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-network-test = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-network-transactions = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-sysinfo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-telemetry = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-tracing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-transaction-pool-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sc-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sp-blockchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -sp-externalities = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-genesis-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-storage = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -sp-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -substrate-build-script-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -substrate-frame-rpc-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -substrate-prometheus-endpoint = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -substrate-test-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -substrate-test-runtime-client = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -substrate-wasm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -try-runtime-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +frame-benchmarking-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +pallet-transaction-payment-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-transaction-payment-rpc-runtime-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sc-basic-authorship = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-block-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-chain-spec = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-client-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-consensus = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-consensus-grandpa = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-consensus-manual-seal = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-consensus-slots = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-network-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-network-sync = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-network-test = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-network-transactions = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-offchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-rpc = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-sysinfo = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-telemetry = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-tracing = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-transaction-pool = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-transaction-pool-api = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sc-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sp-blockchain = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +sp-externalities = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-genesis-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-keystore = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-staking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-storage = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +sp-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +substrate-build-script-utils = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +substrate-frame-rpc-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +substrate-prometheus-endpoint = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +substrate-test-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +substrate-test-runtime-client = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +substrate-wasm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +try-runtime-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } # Polkadot (wasm) -pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-xcm-benchmarks = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -polkadot-core-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -polkadot-node-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -polkadot-parachain-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -polkadot-runtime-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -polkadot-runtime-parachains = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -staging-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -staging-xcm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -staging-xcm-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -westend-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -westend-runtime-constants = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-xcm-benchmarks = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +polkadot-core-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +polkadot-node-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +polkadot-parachain-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +polkadot-runtime-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +polkadot-runtime-parachains = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +staging-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +staging-xcm-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +staging-xcm-executor = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +westend-runtime = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +westend-runtime-constants = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Polkadot (client) -polkadot-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -polkadot-node-subsystem = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -polkadot-overseer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -polkadot-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -polkadot-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } +polkadot-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +polkadot-node-subsystem = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +polkadot-overseer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +polkadot-primitives = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +polkadot-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } # Cumulus (wasm) -cumulus-pallet-dmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-pallet-parachain-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false, features = [ "parameterized-consensus-hook" ] } -cumulus-pallet-session-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-pallet-xcmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-primitives-core = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-primitives-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-primitives-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -parachain-info = { package = "staging-parachain-info", git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -parachains-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +cumulus-pallet-dmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-pallet-parachain-system = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false, features = [ "parameterized-consensus-hook" ] } +cumulus-pallet-session-benchmarking = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-pallet-xcm = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-pallet-xcmp-queue = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-primitives-core = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-primitives-timestamp = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-primitives-utility = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +parachain-info = { package = "staging-parachain-info", git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +parachains-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Cumulus (client) -assets-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -cumulus-client-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-collator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-consensus-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-consensus-proposer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-pov-recovery = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-client-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-primitives-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-relay-chain-interface = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -cumulus-test-relay-sproof-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -emulated-integration-tests-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead" } -xcm-emulator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +assets-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +cumulus-client-cli = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-collator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-consensus-aura = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-consensus-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-consensus-proposer = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-network = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-pov-recovery = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-client-service = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-primitives-parachain-inherent = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-relay-chain-interface = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +cumulus-test-relay-sproof-builder = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } +emulated-integration-tests-common = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0" } +xcm-emulator = { git = "https://github.com/moondance-labs/polkadot-sdk", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Frontier (wasm) -fp-account = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fp-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fp-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fp-self-contained = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-base-fee = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-ethereum = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-chain-id = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-modexp = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-sha3fips = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-evm-precompile-simple = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -pallet-hotfix-sufficients = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -precompile-utils = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fp-account = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fp-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fp-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fp-self-contained = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-base-fee = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-ethereum = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-chain-id = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-modexp = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-sha3fips = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-evm-precompile-simple = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +pallet-hotfix-sufficients = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +precompile-utils = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } # Frontier (client) -fc-api = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fc-cli = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fc-consensus = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fc-db = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fc-mapping-sync = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fc-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", features = [ +fc-api = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-cli = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-consensus = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-db = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-mapping-sync = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-rpc = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", features = [ "rpc-binary-search-estimate", ] } -fc-rpc-core = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } -fc-storage = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0-lookahead", default-features = false } +fc-rpc-core = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } +fc-storage = { git = "https://github.com/moondance-labs/frontier", branch = "tanssi-polkadot-v1.6.0", default-features = false } # General (wasm) bounded-collections = { version = "0.1.8", default-features = false } From 14abe6acd75f03d8f79763c4b5a93cc3164c8d4c Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Thu, 14 Mar 2024 09:44:30 -0700 Subject: [PATCH 20/35] increase timeout in zombie_rotation tests --- test/suites/rotation-para/test_rotation.ts | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/test/suites/rotation-para/test_rotation.ts b/test/suites/rotation-para/test_rotation.ts index 3cd1a9124..842f33bca 100644 --- a/test/suites/rotation-para/test_rotation.ts +++ b/test/suites/rotation-para/test_rotation.ts @@ -90,7 +90,7 @@ describeSuite({ it({ id: "T02", title: "Set 1 collator per parachain, and full_rotation every 5 sessions", - timeout: 60000, + timeout: 240000, test: async function () { const keyring = new Keyring({ type: "sr25519" }); const alice = keyring.addFromUri("//Alice", { name: "Alice default" }); @@ -186,7 +186,7 @@ describeSuite({ it({ id: "T08", title: "Test author noting is correct for both containers", - timeout: 60000, + timeout: 120000, test: async function () { const assignment = await paraApi.query.collatorAssignment.collatorContainerChain(); const paraId2000 = await container2000Api.query.parachainInfo.parachainId(); @@ -231,7 +231,7 @@ describeSuite({ it({ id: "T11", title: "Transactions can be made with ethers", - timeout: 60000, + timeout: 120000, test: async function () { const randomAccount = generateKeyringPair(); const tx = await createTransfer(context, randomAccount.address, 1_000_000_000_000, { @@ -249,7 +249,7 @@ describeSuite({ it({ id: "T12", title: "On session 3 we have 1 collator per chain", - timeout: 120000, + timeout: 240000, test: async function () { await waitToSession(context, paraApi, 3); @@ -269,13 +269,13 @@ describeSuite({ it({ id: "T13", title: "On session 4 collators start syncing the new chains", - timeout: 120000, + timeout: 240000, test: async function () { await waitToSession(context, paraApi, 4); // The node detects assignment when the block is finalized, but "waitSessions" ignores finality. // So wait a few blocks more hoping that the current block will be finalized by then. - await context.waitBlock(3, "Tanssi"); + await context.waitBlock(6, "Tanssi"); const futureAssignment = await paraApi.query.collatorAssignment.pendingCollatorContainerChain(); // The assignment is random, so there is a small chance that it will be the same, // and in that case this test shouldn't fail @@ -324,7 +324,7 @@ describeSuite({ it({ id: "T14", title: "On session 5 collators stop the previously assigned chains", - timeout: 120000, + timeout: 240000, test: async function () { await waitToSession(context, paraApi, 5); const assignment = await paraApi.query.collatorAssignment.collatorContainerChain(); From 9439081826e92e0fce71cfdee2849788722af8d1 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Thu, 14 Mar 2024 12:15:04 -0700 Subject: [PATCH 21/35] add relay config for all zombienet suites --- test/configs/zombieDanceboxUpgrade.json | 15 +++++++++++++++ test/configs/zombieTanssiKeepDb.json | 15 +++++++++++++++ test/configs/zombieTanssiMetrics.json | 15 +++++++++++++++ test/configs/zombieTanssiParathreads.json | 15 +++++++++++++++ test/configs/zombieTanssiRotation.json | 15 +++++++++++++++ test/configs/zombieTanssiWarpSync.json | 15 +++++++++++++++ test/suites/rotation-para/test_rotation.ts | 14 +++++++------- 7 files changed, 97 insertions(+), 7 deletions(-) diff --git a/test/configs/zombieDanceboxUpgrade.json b/test/configs/zombieDanceboxUpgrade.json index 617fc5d03..b844c4d09 100644 --- a/test/configs/zombieDanceboxUpgrade.json +++ b/test/configs/zombieDanceboxUpgrade.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", diff --git a/test/configs/zombieTanssiKeepDb.json b/test/configs/zombieTanssiKeepDb.json index 734ce8412..e9f3a1e20 100644 --- a/test/configs/zombieTanssiKeepDb.json +++ b/test/configs/zombieTanssiKeepDb.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", diff --git a/test/configs/zombieTanssiMetrics.json b/test/configs/zombieTanssiMetrics.json index 4f3406abf..222874e2f 100644 --- a/test/configs/zombieTanssiMetrics.json +++ b/test/configs/zombieTanssiMetrics.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", diff --git a/test/configs/zombieTanssiParathreads.json b/test/configs/zombieTanssiParathreads.json index 9f5bbd928..224e3f87c 100644 --- a/test/configs/zombieTanssiParathreads.json +++ b/test/configs/zombieTanssiParathreads.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", diff --git a/test/configs/zombieTanssiRotation.json b/test/configs/zombieTanssiRotation.json index f5f4af5b5..0d5fee4b8 100644 --- a/test/configs/zombieTanssiRotation.json +++ b/test/configs/zombieTanssiRotation.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", diff --git a/test/configs/zombieTanssiWarpSync.json b/test/configs/zombieTanssiWarpSync.json index 1857ac66a..b019ed671 100644 --- a/test/configs/zombieTanssiWarpSync.json +++ b/test/configs/zombieTanssiWarpSync.json @@ -7,6 +7,21 @@ "chain": "rococo-local", "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], + "genesis": { + "runtimeGenesis":{ + "patch": { + "configuration":{ + "config":{ + "async_backing_params": { + "allowed_ancestry_len": 2, + "max_candidate_depth": 3 + }, + "scheduling_lookahead": 2 + } + } + } + } + }, "nodes": [ { "name": "alice", diff --git a/test/suites/rotation-para/test_rotation.ts b/test/suites/rotation-para/test_rotation.ts index 842f33bca..4b70ee58e 100644 --- a/test/suites/rotation-para/test_rotation.ts +++ b/test/suites/rotation-para/test_rotation.ts @@ -90,7 +90,7 @@ describeSuite({ it({ id: "T02", title: "Set 1 collator per parachain, and full_rotation every 5 sessions", - timeout: 240000, + timeout: 60000, test: async function () { const keyring = new Keyring({ type: "sr25519" }); const alice = keyring.addFromUri("//Alice", { name: "Alice default" }); @@ -186,7 +186,7 @@ describeSuite({ it({ id: "T08", title: "Test author noting is correct for both containers", - timeout: 120000, + timeout: 60000, test: async function () { const assignment = await paraApi.query.collatorAssignment.collatorContainerChain(); const paraId2000 = await container2000Api.query.parachainInfo.parachainId(); @@ -231,7 +231,7 @@ describeSuite({ it({ id: "T11", title: "Transactions can be made with ethers", - timeout: 120000, + timeout: 60000, test: async function () { const randomAccount = generateKeyringPair(); const tx = await createTransfer(context, randomAccount.address, 1_000_000_000_000, { @@ -249,7 +249,7 @@ describeSuite({ it({ id: "T12", title: "On session 3 we have 1 collator per chain", - timeout: 240000, + timeout: 120000, test: async function () { await waitToSession(context, paraApi, 3); @@ -269,7 +269,7 @@ describeSuite({ it({ id: "T13", title: "On session 4 collators start syncing the new chains", - timeout: 240000, + timeout: 120000, test: async function () { await waitToSession(context, paraApi, 4); @@ -324,7 +324,7 @@ describeSuite({ it({ id: "T14", title: "On session 5 collators stop the previously assigned chains", - timeout: 240000, + timeout: 120000, test: async function () { await waitToSession(context, paraApi, 5); const assignment = await paraApi.query.collatorAssignment.collatorContainerChain(); @@ -333,7 +333,7 @@ describeSuite({ // The node detects assignment when the block is finalized, but "waitSessions" ignores finality. // So wait a few blocks more hoping that the current block will be finalized by then. // This also serves to check that Tanssi is producing blocks after the rotation - await context.waitBlock(3, "Tanssi"); + await context.waitBlock(6, "Tanssi"); // First, check that nodes have stopped in their previously assigned chain const oldC2000 = collatorName[assignment3.containerChains[2000][0]]; From 6958b5af3b1b05c01e619602aa5224af81c70df7 Mon Sep 17 00:00:00 2001 From: girazoki Date: Fri, 15 Mar 2024 08:46:01 +0100 Subject: [PATCH 22/35] point at new moonkit commit with expectedBlockTime --- Cargo.lock | 30 +++++++++---------- .../templates/frontier/runtime/src/lib.rs | 1 + .../templates/simple/runtime/src/lib.rs | 1 + runtime/dancebox/src/lib.rs | 1 + runtime/flashbox/src/lib.rs | 1 + 5 files changed, 19 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4c88b40d7..f85dc31b7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -492,7 +492,7 @@ dependencies = [ [[package]] name = "async-backing-primitives" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "sp-api", "sp-consensus-slots", @@ -7118,7 +7118,7 @@ dependencies = [ [[package]] name = "nimbus-consensus" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "async-backing-primitives", "async-trait", @@ -7158,7 +7158,7 @@ dependencies = [ [[package]] name = "nimbus-primitives" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "async-trait", "frame-benchmarking", @@ -7641,7 +7641,7 @@ dependencies = [ [[package]] name = "pallet-async-backing" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-primitives-core", @@ -7661,7 +7661,7 @@ dependencies = [ [[package]] name = "pallet-author-inherent" version = "0.9.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "frame-benchmarking", "frame-support", @@ -8329,7 +8329,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-balances-erc20" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "fp-evm", "frame-support", @@ -8352,7 +8352,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-batch" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "evm", "fp-evm", @@ -8373,7 +8373,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-call-permit" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "evm", "fp-evm", @@ -8424,7 +8424,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompile-xcm-utils" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "fp-evm", "frame-support", @@ -8446,7 +8446,7 @@ dependencies = [ [[package]] name = "pallet-evm-precompileset-assets-erc20" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "fp-evm", "frame-support", @@ -8486,7 +8486,7 @@ dependencies = [ [[package]] name = "pallet-foreign-asset-creator" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "frame-benchmarking", "frame-support", @@ -8658,7 +8658,7 @@ dependencies = [ [[package]] name = "pallet-maintenance-mode" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "cumulus-primitives-core", "frame-support", @@ -8711,7 +8711,7 @@ dependencies = [ [[package]] name = "pallet-migrations" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "frame-benchmarking", "frame-support", @@ -9012,7 +9012,7 @@ dependencies = [ [[package]] name = "pallet-relay-storage-roots" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "cumulus-pallet-parachain-system", "cumulus-primitives-core", @@ -17290,7 +17290,7 @@ dependencies = [ [[package]] name = "xcm-primitives" version = "0.1.0" -source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#9d006f9c3de5d621c6aa54da18fa2c515d0e8341" +source = "git+https://github.com/moondance-labs/moonkit?branch=tanssi-polkadot-v1.6.0#e3da97cf669963ca45c2349f060a3413b39f38d9" dependencies = [ "sp-runtime", ] diff --git a/container-chains/templates/frontier/runtime/src/lib.rs b/container-chains/templates/frontier/runtime/src/lib.rs index deb374435..c06628930 100644 --- a/container-chains/templates/frontier/runtime/src/lib.rs +++ b/container-chains/templates/frontier/runtime/src/lib.rs @@ -546,6 +546,7 @@ impl pallet_async_backing::Config for Runtime { type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; + type ExpectedBlockTime = ConstU64; } impl parachain_info::Config for Runtime {} diff --git a/container-chains/templates/simple/runtime/src/lib.rs b/container-chains/templates/simple/runtime/src/lib.rs index ffc375ba8..09c5c0e4b 100644 --- a/container-chains/templates/simple/runtime/src/lib.rs +++ b/container-chains/templates/simple/runtime/src/lib.rs @@ -436,6 +436,7 @@ impl pallet_async_backing::Config for Runtime { type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; + type ExpectedBlockTime = ConstU64; } impl parachain_info::Config for Runtime {} diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index a24d05252..5b68777ee 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -513,6 +513,7 @@ impl pallet_async_backing::Config for Runtime { type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; + type ExpectedBlockTime = ConstU64; } /// Only callable after `set_validation_data` is called which forms this proof the same way diff --git a/runtime/flashbox/src/lib.rs b/runtime/flashbox/src/lib.rs index 47336e02b..709cc8733 100644 --- a/runtime/flashbox/src/lib.rs +++ b/runtime/flashbox/src/lib.rs @@ -490,6 +490,7 @@ impl pallet_async_backing::Config for Runtime { type AllowMultipleBlocksPerSlot = ConstBool; type GetAndVerifySlot = pallet_async_backing::ParaSlot; + type ExpectedBlockTime = ConstU64; } pub struct OwnApplySession; From 40ea822b8aff98deb53db3555350752eb9c15eaf Mon Sep 17 00:00:00 2001 From: girazoki Date: Fri, 15 Mar 2024 10:03:21 +0100 Subject: [PATCH 23/35] replace with our own custom fork of chopsticks --- test/package.json | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/package.json b/test/package.json index 301565156..5f8bbce54 100644 --- a/test/package.json +++ b/test/package.json @@ -23,7 +23,10 @@ "author": "", "license": "ISC", "devDependencies": { - "@acala-network/chopsticks": "0.9.9-1", + "@acala-network/chopsticks": "npm:@tanssi/chopsticks@^0.9.10-b", + "@acala-network/chopsticks-core": "npm:@tanssi/chopsticks-core@^0.9.10-b", + "@acala-network/chopsticks-db": "npm:@tanssi/chopsticks-db@^0.9.10-b", + "@acala-network/chopsticks-executor": "npm:@tanssi/chopsticks-executor@^0.9.10-b", "@moonbeam-network/api-augment": "0.2600.0", "@moonwall/cli": "5.0.3", "@moonwall/util": "5.0.3", From ef015f6b679efa6cb01444a33cdb881bd85e976a Mon Sep 17 00:00:00 2001 From: girazoki Date: Fri, 15 Mar 2024 10:06:49 +0100 Subject: [PATCH 24/35] update lock --- pnpm-lock.yaml | 256 ++++++++++++++++++++++++++----------------------- 1 file changed, 134 insertions(+), 122 deletions(-) diff --git a/pnpm-lock.yaml b/pnpm-lock.yaml index 681d305df..759bce0e1 100644 --- a/pnpm-lock.yaml +++ b/pnpm-lock.yaml @@ -6,6 +6,15 @@ settings: importers: + .: + dependencies: + api-augment:0.400.0: + specifier: link:@tanssi/api-augment:0.400.0 + version: link:@tanssi/api-augment:0.400.0 + api-augment:latest: + specifier: link:@tanssi/api-augment:latest + version: link:@tanssi/api-augment:latest + test: dependencies: '@zombienet/orchestrator': @@ -19,14 +28,23 @@ importers: version: 0.1.6 devDependencies: '@acala-network/chopsticks': - specifier: 0.9.9-1 - version: 0.9.9-1(debug@4.3.4) + specifier: npm:@tanssi/chopsticks@^0.9.10-b + version: /@tanssi/chopsticks@0.9.10-b(debug@4.3.4) + '@acala-network/chopsticks-core': + specifier: npm:@tanssi/chopsticks-core@^0.9.10-b + version: /@tanssi/chopsticks-core@0.9.10-b + '@acala-network/chopsticks-db': + specifier: npm:@tanssi/chopsticks-db@^0.9.10-b + version: /@tanssi/chopsticks-db@0.9.10-b + '@acala-network/chopsticks-executor': + specifier: npm:@tanssi/chopsticks-executor@^0.9.10-b + version: /@tanssi/chopsticks-executor@0.9.10 '@moonbeam-network/api-augment': specifier: 0.2600.0 version: 0.2600.0 '@moonwall/cli': specifier: 5.0.3 - version: 5.0.3(@acala-network/chopsticks@0.9.9-1)(@polkadot/api@10.11.2)(@types/node@20.10.5)(@vitest/ui@1.1.0)(typescript@5.3.3)(vitest@1.1.0) + version: 5.0.3(@polkadot/api@10.11.2)(@tanssi/chopsticks@0.9.10-b)(@types/node@20.10.5)(@vitest/ui@1.1.0)(typescript@5.3.3)(vitest@1.1.0) '@moonwall/util': specifier: 5.0.3 version: 5.0.3(@polkadot/api@10.11.2)(typescript@5.3.3)(vitest@1.1.0) @@ -189,112 +207,6 @@ packages: engines: {node: '>=0.10.0'} dev: true - /@acala-network/chopsticks-core@0.9.9-1: - resolution: {integrity: sha512-MYcAt9VdDNYgS4XHv2yB8XKSUfBxFbxjEQqg2eITqv3x2eu9p1xTDTeOVDLozjutJXdzfWqEYOGy+3AQehQzEA==} - dependencies: - '@acala-network/chopsticks-executor': 0.9.9-1 - '@polkadot/rpc-provider': 10.11.2 - '@polkadot/types': 10.11.2 - '@polkadot/types-codec': 10.11.2 - '@polkadot/types-known': 10.11.2 - '@polkadot/util': 12.6.2 - '@polkadot/util-crypto': 12.6.2(@polkadot/util@12.6.2) - comlink: 4.4.1 - eventemitter3: 5.0.1 - lodash: 4.17.21 - lru-cache: 10.1.0 - pino: 8.18.0 - pino-pretty: 10.3.1 - rxjs: 7.8.1 - zod: 3.22.4 - transitivePeerDependencies: - - bufferutil - - supports-color - - utf-8-validate - dev: true - - /@acala-network/chopsticks-db@0.9.9-1: - resolution: {integrity: sha512-c/a+LduCyF6GZGVY39dnbjiFaAKOoF9MJ22LJqnyAGh7jDmJpasFnUsxBTgwm5rVxBEjDk2CXpBulBf+IebXvw==} - dependencies: - '@acala-network/chopsticks-core': 0.9.9-1 - '@polkadot/util': 12.6.2 - idb: 8.0.0 - sqlite3: 5.1.7 - typeorm: 0.3.20(sqlite3@5.1.7) - transitivePeerDependencies: - - '@google-cloud/spanner' - - '@sap/hana-client' - - better-sqlite3 - - bluebird - - bufferutil - - hdb-pool - - ioredis - - mongodb - - mssql - - mysql2 - - oracledb - - pg - - pg-native - - pg-query-stream - - redis - - sql.js - - supports-color - - ts-node - - typeorm-aurora-data-api-driver - - utf-8-validate - dev: true - - /@acala-network/chopsticks-executor@0.9.9-1: - resolution: {integrity: sha512-uuAfvvyknDUuAp3WpvaPXDcIdD0K98c/UdP0sh3AicYp6VCH17INVLLnOdiPq+/TMHDSJ599J6FZYGBcN27QBA==} - dependencies: - '@polkadot/util': 12.6.2 - '@polkadot/wasm-util': 7.3.2(@polkadot/util@12.6.2) - dev: true - - /@acala-network/chopsticks@0.9.9-1(debug@4.3.4): - resolution: {integrity: sha512-7Ejpv0DEj8go9zhxxkWKAKQArMHqkdryjRJdaSXZ5beDbrMlTOqvzOTLSp2EnP+tjyoG0f+L6bxEqN+6R3mmKg==} - hasBin: true - dependencies: - '@acala-network/chopsticks-core': 0.9.9-1 - '@acala-network/chopsticks-db': 0.9.9-1 - '@pnpm/npm-conf': 2.2.2 - '@polkadot/api-augment': 10.11.2 - '@polkadot/types': 10.11.2 - '@polkadot/util': 12.6.2 - '@polkadot/util-crypto': 12.6.2(@polkadot/util@12.6.2) - axios: 1.6.7(debug@4.3.4) - dotenv: 16.4.1 - global-agent: 3.0.0 - js-yaml: 4.1.0 - jsondiffpatch: 0.5.0 - lodash: 4.17.21 - ws: 8.16.0 - yargs: 17.7.2 - zod: 3.22.4 - transitivePeerDependencies: - - '@google-cloud/spanner' - - '@sap/hana-client' - - better-sqlite3 - - bluebird - - bufferutil - - debug - - hdb-pool - - ioredis - - mongodb - - mssql - - mysql2 - - oracledb - - pg - - pg-native - - pg-query-stream - - redis - - sql.js - - supports-color - - ts-node - - typeorm-aurora-data-api-driver - - utf-8-validate - dev: true - /@adraffy/ens-normalize@1.10.0: resolution: {integrity: sha512-nA9XHtlAkYfJxY7bce8DcN7eKxWWCWkU+1GR9d+U6MbNpfwQp8TI7vqOsBsMcHoT4mBu2kypKoSKnghEzOOq5Q==} dev: true @@ -818,7 +730,7 @@ packages: engines: {node: '>=14.0.0'} dev: true - /@moonwall/cli@5.0.3(@acala-network/chopsticks@0.9.9-1)(@polkadot/api@10.11.2)(@types/node@20.10.5)(@vitest/ui@1.1.0)(typescript@5.3.3)(vitest@1.1.0): + /@moonwall/cli@5.0.3(@polkadot/api@10.11.2)(@tanssi/chopsticks@0.9.10-b)(@types/node@20.10.5)(@vitest/ui@1.1.0)(typescript@5.3.3)(vitest@1.1.0): resolution: {integrity: sha512-n9j6Er3ysUa1RgJlktZYCi7DiJpEdGmG/nhOAB0b3w/k9S4qDs53c29Bk6brXR9uLocROmH3qVAufR7iy9jlnQ==} engines: {node: '>=20', pnpm: '>=7'} hasBin: true @@ -828,7 +740,7 @@ packages: '@vitest/ui': 1.2.2 vitest: 1.2.2 dependencies: - '@acala-network/chopsticks': 0.9.9-1(debug@4.3.4) + '@acala-network/chopsticks': /@tanssi/chopsticks@0.9.10-b(debug@4.3.4) '@moonbeam-network/api-augment': 0.2700.0 '@moonwall/types': 5.0.3(@polkadot/api@10.11.2)(typescript@5.3.3) '@moonwall/util': 5.0.3(@polkadot/api@10.11.2)(typescript@5.3.3)(vitest@1.1.0) @@ -1035,7 +947,7 @@ packages: requiresBuild: true dependencies: '@gar/promisify': 1.1.3 - semver: 7.5.4 + semver: 7.6.0 dev: true optional: true @@ -1784,6 +1696,112 @@ packages: /@substrate/ss58-registry@1.44.0: resolution: {integrity: sha512-7lQ/7mMCzVNSEfDS4BCqnRnKCFKpcOaPrxMeGTXHX1YQzM/m2BBHjbK2C3dJvjv7GYxMiaTq/HdWQj1xS6ss+A==} + /@tanssi/chopsticks-core@0.9.10-b: + resolution: {integrity: sha512-5sBlEpdwqz+rNkgoOQTeZGrWK5zRl9oBJagomPf1YF7x/eRdFvRLBnTdiZse46kcF6J+zFH3m3tfT9FDjOcexg==} + dependencies: + '@polkadot/rpc-provider': 10.11.2 + '@polkadot/types': 10.11.2 + '@polkadot/types-codec': 10.11.2 + '@polkadot/types-known': 10.11.2 + '@polkadot/util': 12.6.2 + '@polkadot/util-crypto': 12.6.2(@polkadot/util@12.6.2) + '@tanssi/chopsticks-executor': 0.9.10 + comlink: 4.4.1 + eventemitter3: 5.0.1 + lodash: 4.17.21 + lru-cache: 10.1.0 + pino: 8.18.0 + pino-pretty: 10.3.1 + rxjs: 7.8.1 + zod: 3.22.4 + transitivePeerDependencies: + - bufferutil + - supports-color + - utf-8-validate + dev: true + + /@tanssi/chopsticks-db@0.9.10-b: + resolution: {integrity: sha512-z58nJJxmUnRO6gzFm3HaQfEtv5g3eXp9Dv7z0bkwfJ8tSjip21NKByMOKOuTEAXwq/HMj7uBTa9f6NZ2P9po/w==} + dependencies: + '@polkadot/util': 12.6.2 + '@tanssi/chopsticks-core': 0.9.10-b + idb: 8.0.0 + sqlite3: 5.1.7 + typeorm: 0.3.20(sqlite3@5.1.7) + transitivePeerDependencies: + - '@google-cloud/spanner' + - '@sap/hana-client' + - better-sqlite3 + - bluebird + - bufferutil + - hdb-pool + - ioredis + - mongodb + - mssql + - mysql2 + - oracledb + - pg + - pg-native + - pg-query-stream + - redis + - sql.js + - supports-color + - ts-node + - typeorm-aurora-data-api-driver + - utf-8-validate + dev: true + + /@tanssi/chopsticks-executor@0.9.10: + resolution: {integrity: sha512-bLrWgiS8ARhA337Jl1Qx+KRca7r30GurGi85ZtO11QZB9TDw+ZTLmF2VQ92Eo9aQPXQoEeo4w0+buq3Jbd9VyA==} + dependencies: + '@polkadot/util': 12.6.2 + '@polkadot/wasm-util': 7.3.2(@polkadot/util@12.6.2) + dev: true + + /@tanssi/chopsticks@0.9.10-b(debug@4.3.4): + resolution: {integrity: sha512-7uWWwCLg5uvJq34UXUgj2sWDLgEUr6OaKhJ2ZnTPqV0A8Tf1zB+t1QwS0uzpAgUIV2l60LMYzoDex2fp2sgUDA==} + hasBin: true + dependencies: + '@pnpm/npm-conf': 2.2.2 + '@polkadot/api-augment': 10.11.2 + '@polkadot/types': 10.11.2 + '@polkadot/util': 12.6.2 + '@polkadot/util-crypto': 12.6.2(@polkadot/util@12.6.2) + '@tanssi/chopsticks-core': 0.9.10-b + '@tanssi/chopsticks-db': 0.9.10-b + axios: 1.6.7(debug@4.3.4) + dotenv: 16.4.4 + global-agent: 3.0.0 + js-yaml: 4.1.0 + jsondiffpatch: 0.5.0 + lodash: 4.17.21 + ws: 8.16.0 + yargs: 17.7.2 + zod: 3.22.4 + transitivePeerDependencies: + - '@google-cloud/spanner' + - '@sap/hana-client' + - better-sqlite3 + - bluebird + - bufferutil + - debug + - hdb-pool + - ioredis + - mongodb + - mssql + - mysql2 + - oracledb + - pg + - pg-native + - pg-query-stream + - redis + - sql.js + - supports-color + - ts-node + - typeorm-aurora-data-api-driver + - utf-8-validate + dev: true + /@tootallnate/once@1.1.2: resolution: {integrity: sha512-RbzJvlNzmRq5c3O09UipeuXno4tA1FE6ikOjxZK0tuxVv3412l64l5t1W5pj4+rJq9vpkm/kwiR07aZXnsKPxw==} engines: {node: '>= 6'} @@ -2975,11 +2993,6 @@ packages: esutils: 2.0.3 dev: true - /dotenv@16.4.1: - resolution: {integrity: sha512-CjA3y+Dr3FyFDOAMnxZEGtnW9KBR2M0JvvUtXNW+dYJL5ROWxP9DUHCwgFqpMk0OXCc0ljhaNTr2w/kutYIcHQ==} - engines: {node: '>=12'} - dev: true - /dotenv@16.4.4: resolution: {integrity: sha512-XvPXc8XAQThSjAbY6cQ/9PcBXmFoWuw1sQ3b8HqUCR6ziGXjkTi//kB9SWa2UwqlgdAIuRqAa/9hVljzPehbYg==} engines: {node: '>=12'} @@ -3656,7 +3669,7 @@ packages: es6-error: 4.1.1 matcher: 3.0.0 roarr: 2.15.4 - semver: 7.5.4 + semver: 7.6.0 serialize-error: 7.0.1 dev: true @@ -4337,7 +4350,6 @@ packages: chalk: 3.0.0 diff-match-patch: 1.0.5 dev: true - bundledDependencies: [] /jsonfile@6.1.0: resolution: {integrity: sha512-5dgndWOriYSm5cnYaJNhalLNDKOqFwyDB/rr1E9ZsGciGvKPs8R2xYGCacuf3z6K1YKDz182fd+fY3cn3pMqXQ==} @@ -5009,7 +5021,7 @@ packages: resolution: {integrity: sha512-p7eGEiQil0YUV3ItH4/tBb781L5impVmmx2E9FRKF7d18XXzp4PGT2tdYMFY6wQqgxD0IwNZOiSJ0/K0fSi/OA==} engines: {node: '>=10'} dependencies: - semver: 7.5.4 + semver: 7.6.0 dev: true /node-addon-api@7.1.0: @@ -5058,7 +5070,7 @@ packages: nopt: 5.0.0 npmlog: 6.0.2 rimraf: 3.0.2 - semver: 7.5.4 + semver: 7.6.0 tar: 6.1.15 which: 2.0.2 transitivePeerDependencies: @@ -6385,7 +6397,7 @@ packages: cli-highlight: 2.1.11 dayjs: 1.11.10 debug: 4.3.4(supports-color@8.1.1) - dotenv: 16.4.1 + dotenv: 16.4.4 glob: 10.3.10 mkdirp: 2.1.6 reflect-metadata: 0.2.1 From c2f8050ded0af95b2dabe2784bc38127b9e2cd28 Mon Sep 17 00:00:00 2001 From: girazoki Date: Fri, 15 Mar 2024 11:32:08 +0100 Subject: [PATCH 25/35] always ask the runtime for the slot duration, otherwise there will be downtime and node needs to reboot --- node/src/service.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/node/src/service.rs b/node/src/service.rs index 3cf6e6208..67e89ff3c 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -905,6 +905,12 @@ fn start_consensus_orchestrator( ) .await; + // Fetch duration every block to avoid downtime when passing from 12 to 6s + let slot_duration = cumulus_client_consensus_aura::slot_duration( + &*client_set_aside_for_cidp.clone(), + ) + .expect("Slot duration should be set"); + let timestamp = sp_timestamp::InherentDataProvider::from_system_time(); let slot = From 4ec28de102886992fbb0fbf44e4bb6f8bf5b4ea3 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 15 Mar 2024 06:40:47 -0700 Subject: [PATCH 26/35] fix staking ts tests --- test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts | 2 +- .../dev-tanssi/staking/test_staking_rewards_non_balanced.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts b/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts index 07d80f50b..ac68ee3a3 100644 --- a/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts +++ b/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts @@ -63,7 +63,7 @@ describeSuite({ const events = await polkadotJs.query.system.events(); const issuance = await fetchIssuance(events).amount.toBigInt(); const chainRewards = (issuance * 7n) / 10n; - const expectedOrchestratorReward = chainRewards / 3n; + const expectedOrchestratorReward = chainRewards - chainRewards * 2n / 3n; const reward = await fetchRewardAuthorOrchestrator(events); const stakingRewardedCollator = await filterRewardStakingCollator(events, reward.accountId.toString()); const stakingRewardedDelegators = await filterRewardStakingDelegators( diff --git a/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts b/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts index 7b57ee5c5..edab0f441 100644 --- a/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts +++ b/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts @@ -63,7 +63,7 @@ describeSuite({ const events = await polkadotJs.query.system.events(); const issuance = await fetchIssuance(events).amount.toBigInt(); const chainRewards = (issuance * 7n) / 10n; - const expectedOrchestratorReward = chainRewards / 3n; + const expectedOrchestratorReward = chainRewards - chainRewards * 2n / 3n; const reward = await fetchRewardAuthorOrchestrator(events); const stakingRewardedCollator = await filterRewardStakingCollator(events, reward.accountId.toString()); const stakingRewardedDelegators = await filterRewardStakingDelegators( From 2178a1512f0edea4d0d79982ecd5080021075eac Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 15 Mar 2024 06:43:39 -0700 Subject: [PATCH 27/35] update api-augment --- .../src/dancebox/interfaces/augment-api-consts.ts | 6 ++++++ .../src/flashbox/interfaces/augment-api-consts.ts | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/typescript-api/src/dancebox/interfaces/augment-api-consts.ts b/typescript-api/src/dancebox/interfaces/augment-api-consts.ts index 764c02f70..c363fe341 100644 --- a/typescript-api/src/dancebox/interfaces/augment-api-consts.ts +++ b/typescript-api/src/dancebox/interfaces/augment-api-consts.ts @@ -22,6 +22,12 @@ export type __AugmentedConst = AugmentedConst declare module "@polkadot/api-base/types/consts" { interface AugmentedConsts { + asyncBacking: { + /** Purely informative, but used by mocking tools like chospticks to allow knowing how to mock blocks */ + expectedBlockTime: u64 & AugmentedConst; + /** Generic const */ + [key: string]: Codec; + }; balances: { /** * The minimum amount required to keep an account open. MUST BE GREATER THAN ZERO! diff --git a/typescript-api/src/flashbox/interfaces/augment-api-consts.ts b/typescript-api/src/flashbox/interfaces/augment-api-consts.ts index 2aeec8897..b2c29c3ef 100644 --- a/typescript-api/src/flashbox/interfaces/augment-api-consts.ts +++ b/typescript-api/src/flashbox/interfaces/augment-api-consts.ts @@ -21,6 +21,12 @@ export type __AugmentedConst = AugmentedConst declare module "@polkadot/api-base/types/consts" { interface AugmentedConsts { + asyncBacking: { + /** Purely informative, but used by mocking tools like chospticks to allow knowing how to mock blocks */ + expectedBlockTime: u64 & AugmentedConst; + /** Generic const */ + [key: string]: Codec; + }; balances: { /** * The minimum amount required to keep an account open. MUST BE GREATER THAN ZERO! From 4c40acfce876d28680ff9ac712bb52a8d3de25fa Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 15 Mar 2024 06:44:42 -0700 Subject: [PATCH 28/35] use previous timeout in zombie_parathreads --- test/suites/parathreads/test_tanssi_parathreads.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/suites/parathreads/test_tanssi_parathreads.ts b/test/suites/parathreads/test_tanssi_parathreads.ts index 415ab9119..5824145ab 100644 --- a/test/suites/parathreads/test_tanssi_parathreads.ts +++ b/test/suites/parathreads/test_tanssi_parathreads.ts @@ -258,7 +258,7 @@ describeSuite({ it({ id: "T12", title: "Check block frequency of parathreads", - timeout: 240000, + timeout: 120000, test: async function () { // Wait 1 session so that parathreads have produced at least a few blocks each await waitSessions(context, paraApi, 2); From 1f1f424ffa1ee0886866d058e5a7241f6d0e6840 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 15 Mar 2024 06:57:49 -0700 Subject: [PATCH 29/35] fmt --- test/configs/zombieDanceboxUpgrade.json | 6 +++--- test/configs/zombieTanssi.json | 6 +++--- test/configs/zombieTanssiKeepDb.json | 6 +++--- test/configs/zombieTanssiMetrics.json | 6 +++--- test/configs/zombieTanssiParathreads.json | 6 +++--- test/configs/zombieTanssiRotation.json | 6 +++--- test/configs/zombieTanssiWarpSync.json | 6 +++--- .../dev-tanssi/staking/test_staking_rewards_balanced.ts | 2 +- .../dev-tanssi/staking/test_staking_rewards_non_balanced.ts | 2 +- 9 files changed, 23 insertions(+), 23 deletions(-) diff --git a/test/configs/zombieDanceboxUpgrade.json b/test/configs/zombieDanceboxUpgrade.json index b844c4d09..0ac13d6db 100644 --- a/test/configs/zombieDanceboxUpgrade.json +++ b/test/configs/zombieDanceboxUpgrade.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/configs/zombieTanssi.json b/test/configs/zombieTanssi.json index 34cbe6bad..b5998030a 100644 --- a/test/configs/zombieTanssi.json +++ b/test/configs/zombieTanssi.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/configs/zombieTanssiKeepDb.json b/test/configs/zombieTanssiKeepDb.json index e9f3a1e20..0e7e5e980 100644 --- a/test/configs/zombieTanssiKeepDb.json +++ b/test/configs/zombieTanssiKeepDb.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/configs/zombieTanssiMetrics.json b/test/configs/zombieTanssiMetrics.json index 222874e2f..c332f6e39 100644 --- a/test/configs/zombieTanssiMetrics.json +++ b/test/configs/zombieTanssiMetrics.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/configs/zombieTanssiParathreads.json b/test/configs/zombieTanssiParathreads.json index 224e3f87c..b70c2459f 100644 --- a/test/configs/zombieTanssiParathreads.json +++ b/test/configs/zombieTanssiParathreads.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/configs/zombieTanssiRotation.json b/test/configs/zombieTanssiRotation.json index 0d5fee4b8..3b92182da 100644 --- a/test/configs/zombieTanssiRotation.json +++ b/test/configs/zombieTanssiRotation.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/configs/zombieTanssiWarpSync.json b/test/configs/zombieTanssiWarpSync.json index b019ed671..1b6d94cfd 100644 --- a/test/configs/zombieTanssiWarpSync.json +++ b/test/configs/zombieTanssiWarpSync.json @@ -8,10 +8,10 @@ "default_command": "tmp/polkadot", "default_args": ["--no-hardware-benchmarks", "-lparachain=debug", "--database=paritydb", "--no-beefy"], "genesis": { - "runtimeGenesis":{ + "runtimeGenesis": { "patch": { - "configuration":{ - "config":{ + "configuration": { + "config": { "async_backing_params": { "allowed_ancestry_len": 2, "max_candidate_depth": 3 diff --git a/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts b/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts index ac68ee3a3..b7bb4c127 100644 --- a/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts +++ b/test/suites/dev-tanssi/staking/test_staking_rewards_balanced.ts @@ -63,7 +63,7 @@ describeSuite({ const events = await polkadotJs.query.system.events(); const issuance = await fetchIssuance(events).amount.toBigInt(); const chainRewards = (issuance * 7n) / 10n; - const expectedOrchestratorReward = chainRewards - chainRewards * 2n / 3n; + const expectedOrchestratorReward = chainRewards - (chainRewards * 2n) / 3n; const reward = await fetchRewardAuthorOrchestrator(events); const stakingRewardedCollator = await filterRewardStakingCollator(events, reward.accountId.toString()); const stakingRewardedDelegators = await filterRewardStakingDelegators( diff --git a/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts b/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts index edab0f441..12efeaf3e 100644 --- a/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts +++ b/test/suites/dev-tanssi/staking/test_staking_rewards_non_balanced.ts @@ -63,7 +63,7 @@ describeSuite({ const events = await polkadotJs.query.system.events(); const issuance = await fetchIssuance(events).amount.toBigInt(); const chainRewards = (issuance * 7n) / 10n; - const expectedOrchestratorReward = chainRewards - chainRewards * 2n / 3n; + const expectedOrchestratorReward = chainRewards - (chainRewards * 2n) / 3n; const reward = await fetchRewardAuthorOrchestrator(events); const stakingRewardedCollator = await filterRewardStakingCollator(events, reward.accountId.toString()); const stakingRewardedDelegators = await filterRewardStakingDelegators( From 2a3a3ce412327b336505f12b7c3b082f429220bf Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 15 Mar 2024 07:03:49 -0700 Subject: [PATCH 30/35] use proper Receiver --- node/src/service.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/node/src/service.rs b/node/src/service.rs index 67e89ff3c..d5cc36030 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -81,7 +81,7 @@ use { }, tokio::sync::{ mpsc::{unbounded_channel, UnboundedSender}, - watch::Receiver, + watch, }, }; @@ -861,7 +861,7 @@ fn start_consensus_orchestrator( overseer_handle: OverseerHandle, announce_block: Arc>) + Send + Sync>, proposer_factory: ParachainProposerFactory, - end_lookahead_receiver: Option>, + end_lookahead_receiver: Option>, ) { let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client) .expect("start_consensus_orchestrator: slot duration should exist"); From 26cdc942d00a8a150bb98791e2ccb8207c35945c Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Fri, 15 Mar 2024 07:12:33 -0700 Subject: [PATCH 31/35] cleanup some comments --- node/src/service.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/node/src/service.rs b/node/src/service.rs index d5cc36030..b525210b6 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -839,7 +839,6 @@ fn start_consensus_container( para_backend: backend, code_hash_provider, end_lookahead_receiver: None, - //collation_request_receiver: None, }; let fut = lookahead_tanssi_aura::run::(params); @@ -977,10 +976,8 @@ fn start_consensus_orchestrator( code_hash_provider, para_backend: backend, end_lookahead_receiver, - //collation_request_receiver: None, }; - //let fut = basic_tanssi_aura::run::(params); let fut = lookahead_tanssi_aura::run::(params); spawner.spawn("tanssi-aura", None, fut); } From 6c51a8ece2d91d2adcfcb8ebf6381ab2af205e01 Mon Sep 17 00:00:00 2001 From: Tomasz Polaczyk Date: Mon, 18 Mar 2024 12:48:12 +0100 Subject: [PATCH 32/35] Read aura slot duration at parent block, not best block Fixes a rare bug where collators will try to build a block before the runtime upgrade using the slot duration from the latest block, which is after the runtime upgrade, resulting in a stall --- Cargo.lock | 1 + node/Cargo.toml | 1 + node/src/service.rs | 8 ++++++-- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f85dc31b7..b2f528d12 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -15312,6 +15312,7 @@ dependencies = [ "sc-cli", "sc-client-api", "sc-consensus", + "sc-consensus-aura", "sc-consensus-manual-seal", "sc-executor", "sc-network", diff --git a/node/Cargo.toml b/node/Cargo.toml index 058090d3d..1299e92fb 100644 --- a/node/Cargo.toml +++ b/node/Cargo.toml @@ -52,6 +52,7 @@ sc-chain-spec = { workspace = true } sc-cli = { workspace = true } sc-client-api = { workspace = true } sc-consensus = { workspace = true } +sc-consensus-aura = { workspace = true } sc-consensus-manual-seal = { workspace = true } sc-executor = { workspace = true } sc-network = { workspace = true } diff --git a/node/src/service.rs b/node/src/service.rs index b525210b6..a39edaf03 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -905,8 +905,9 @@ fn start_consensus_orchestrator( .await; // Fetch duration every block to avoid downtime when passing from 12 to 6s - let slot_duration = cumulus_client_consensus_aura::slot_duration( + let slot_duration = sc_consensus_aura::standalone::slot_duration_at( &*client_set_aside_for_cidp.clone(), + block_hash, ) .expect("Slot duration should be set"); @@ -1085,7 +1086,10 @@ pub fn start_dev_node( let para_head_key = RelayWellKnownKeys::para_head(para_id); let relay_slot_key = RelayWellKnownKeys::CURRENT_SLOT.to_vec(); - let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client.clone()).expect("Slot duration should be set"); + let slot_duration = sc_consensus_aura::standalone::slot_duration_at( + &*client.clone(), + block, + ).expect("Slot duration should be set"); let mut timestamp: u64 = 0u64; TIMESTAMP.with(|x| { From ae454219e564e3eb7850605ee9d465faee05087e Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 19 Mar 2024 09:09:33 -0700 Subject: [PATCH 33/35] adjust inflation --- runtime/dancebox/src/lib.rs | 6 +++--- runtime/flashbox/src/lib.rs | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/runtime/dancebox/src/lib.rs b/runtime/dancebox/src/lib.rs index 3da14ec0a..6c8cbc0ba 100644 --- a/runtime/dancebox/src/lib.rs +++ b/runtime/dancebox/src/lib.rs @@ -1321,12 +1321,12 @@ parameter_types! { pub ParachainBondAccount: AccountId32 = PalletId(*b"ParaBond").into_account_truncating(); pub PendingRewardsAccount: AccountId32 = PalletId(*b"PENDREWD").into_account_truncating(); // The equation to solve is: - // initial_supply * (1.05) = initial_supply * (1+x)^2_629_800 - // we should solve for x = (1.05)^(1/2_629_800) -1 -> 0.000000019 per block or 19/1_000_000_000 + // initial_supply * (1.05) = initial_supply * (1+x)^5_259_600 + // we should solve for x = (1.05)^(1/5_259_600) -1 -> 0.000000009 per block or 9/1_000_000_000 // 1% in the case of dev mode // TODO: check if we can put the prod inflation for tests too // TODO: better calculus for going from annual to block inflation (if it can be done) - pub const InflationRate: Perbill = prod_or_fast!(Perbill::from_parts(19), Perbill::from_percent(1)); + pub const InflationRate: Perbill = prod_or_fast!(Perbill::from_parts(9), Perbill::from_percent(1)); // 30% for parachain bond, so 70% for staking pub const RewardsPortion: Perbill = Perbill::from_percent(70); diff --git a/runtime/flashbox/src/lib.rs b/runtime/flashbox/src/lib.rs index 0a8a8d8d8..ec156dc99 100644 --- a/runtime/flashbox/src/lib.rs +++ b/runtime/flashbox/src/lib.rs @@ -1081,12 +1081,12 @@ parameter_types! { pub ParachainBondAccount: AccountId32 = PalletId(*b"ParaBond").into_account_truncating(); pub PendingRewardsAccount: AccountId32 = PalletId(*b"PENDREWD").into_account_truncating(); // The equation to solve is: - // initial_supply * (1.05) = initial_supply * (1+x)^2_629_800 - // we should solve for x = (1.05)^(1/2_629_800) -1 -> 0.000000019 per block or 19/1_000_000_000 + // initial_supply * (1.05) = initial_supply * (1+x)^5_259_600 + // we should solve for x = (1.05)^(1/5_259_600) -1 -> 0.000000009 per block or 9/1_000_000_000 // 1% in the case of dev mode // TODO: check if we can put the prod inflation for tests too // TODO: better calculus for going from annual to block inflation (if it can be done) - pub const InflationRate: Perbill = prod_or_fast!(Perbill::from_parts(19), Perbill::from_percent(1)); + pub const InflationRate: Perbill = prod_or_fast!(Perbill::from_parts(9), Perbill::from_percent(1)); // 30% for parachain bond, so 70% for staking pub const RewardsPortion: Perbill = Perbill::from_percent(70); From 19a54c5601d00d59b7497e10cd2d3e88aa5fcc89 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 19 Mar 2024 11:02:55 -0700 Subject: [PATCH 34/35] pr comments --- node/src/service.rs | 3 ++- test/configs/zombieTanssi.json | 3 --- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/node/src/service.rs b/node/src/service.rs index fd33cdfdd..51c39a2c5 100644 --- a/node/src/service.rs +++ b/node/src/service.rs @@ -76,7 +76,6 @@ use { collators::lookahead::{ self as lookahead_tanssi_aura, Params as LookaheadTanssiAuraParams, }, - //collators::basic::{self as basic_tanssi_aura, Params as BasicTanssiAuraParams}, OrchestratorAuraWorkerAuxData, }, tokio::sync::{ @@ -404,6 +403,8 @@ async fn start_node_impl( sync_service: node_builder.network.sync_service.clone(), })?; + // This channel allows us to notify the lookahead collator when it should stop. + // Useful when rotating containers. let (end_lookahead_sender, end_lookahead_receiver) = tokio::sync::watch::channel(()); if validator { diff --git a/test/configs/zombieTanssi.json b/test/configs/zombieTanssi.json index b5998030a..aabcf89df 100644 --- a/test/configs/zombieTanssi.json +++ b/test/configs/zombieTanssi.json @@ -61,7 +61,6 @@ }, { "name": "Collator2000-01", - "ws_port": "9960", "command": "../target/release/tanssi-node", "args": ["--no-hardware-benchmarks", "--database=paritydb", "--wasmtime-precompiled=wasm"] }, @@ -72,7 +71,6 @@ }, { "name": "Collator2001-01", - "ws_port": "9961", "command": "../target/release/tanssi-node", "args": ["--no-hardware-benchmarks", "--database=paritydb", "--wasmtime-precompiled=wasm"] }, @@ -83,7 +81,6 @@ }, { "name": "Collator2002-01", - "ws_port": "9962", "command": "../target/release/tanssi-node", "args": ["--no-hardware-benchmarks", "--database=paritydb", "--wasmtime-precompiled=wasm"] }, From 053245b73bff17e76bc82d2d2e19d52bbcbd6063 Mon Sep 17 00:00:00 2001 From: Agusrodri Date: Tue, 19 Mar 2024 11:08:52 -0700 Subject: [PATCH 35/35] refactor imports --- client/consensus/src/collators/lookahead.rs | 74 ++++++++++----------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/client/consensus/src/collators/lookahead.rs b/client/consensus/src/collators/lookahead.rs index 780dad247..27b15fe12 100644 --- a/client/consensus/src/collators/lookahead.rs +++ b/client/consensus/src/collators/lookahead.rs @@ -31,43 +31,43 @@ //! The main limitation is block propagation time - i.e. the new blocks created by an author //! must be propagated to the next author before their turn. -use async_backing_primitives::UnincludedSegmentApi; -use cumulus_client_collator::service::ServiceInterface as CollatorServiceInterface; -use cumulus_client_consensus_common::{ - self as consensus_common, load_abridged_host_configuration, ParachainBlockImportMarker, - ParentSearchParams, -}; -use cumulus_client_consensus_proposer::ProposerInterface; -use cumulus_primitives_core::{relay_chain::Hash as PHash, PersistedValidationData}; -use cumulus_relay_chain_interface::RelayChainInterface; -use parity_scale_codec::{Codec, Encode}; -use polkadot_node_primitives::SubmitCollationParams; -use polkadot_node_subsystem::messages::{ - CollationGenerationMessage, RuntimeApiMessage, RuntimeApiRequest, -}; -use polkadot_overseer::Handle as OverseerHandle; -use polkadot_primitives::{CollatorPair, Id as ParaId, OccupiedCoreAssumption}; - -use futures::{channel::oneshot, prelude::*}; -use sc_client_api::{backend::AuxStore, BlockBackend, BlockOf}; -use sc_consensus::BlockImport; -use sc_consensus_slots::InherentDataProviderExt; -use sp_api::ProvideRuntimeApi; -use sp_application_crypto::AppPublic; -use sp_blockchain::HeaderBackend; -use sp_consensus::SyncOracle; -use sp_consensus_aura::{Slot, SlotDuration}; -use sp_core::crypto::Pair; -use sp_inherents::CreateInherentDataProviders; -use sp_keystore::KeystorePtr; -use sp_runtime::traits::{Block as BlockT, Header as HeaderT, Member}; -use std::{convert::TryFrom, error::Error, sync::Arc, time::Duration}; -use tokio::sync::watch::Receiver; - -use crate::{ - collators::{self as collator_util, tanssi_claim_slot, SlotClaim}, - consensus_orchestrator::RetrieveAuthoritiesFromOrchestrator, - OrchestratorAuraWorkerAuxData, +use { + crate::{ + collators::{self as collator_util, tanssi_claim_slot, SlotClaim}, + consensus_orchestrator::RetrieveAuthoritiesFromOrchestrator, + OrchestratorAuraWorkerAuxData, + }, + async_backing_primitives::UnincludedSegmentApi, + cumulus_client_collator::service::ServiceInterface as CollatorServiceInterface, + cumulus_client_consensus_common::{ + self as consensus_common, load_abridged_host_configuration, ParachainBlockImportMarker, + ParentSearchParams, + }, + cumulus_client_consensus_proposer::ProposerInterface, + cumulus_primitives_core::{relay_chain::Hash as PHash, PersistedValidationData}, + cumulus_relay_chain_interface::RelayChainInterface, + futures::{channel::oneshot, prelude::*}, + parity_scale_codec::{Codec, Encode}, + polkadot_node_primitives::SubmitCollationParams, + polkadot_node_subsystem::messages::{ + CollationGenerationMessage, RuntimeApiMessage, RuntimeApiRequest, + }, + polkadot_overseer::Handle as OverseerHandle, + polkadot_primitives::{CollatorPair, Id as ParaId, OccupiedCoreAssumption}, + sc_client_api::{backend::AuxStore, BlockBackend, BlockOf}, + sc_consensus::BlockImport, + sc_consensus_slots::InherentDataProviderExt, + sp_api::ProvideRuntimeApi, + sp_application_crypto::AppPublic, + sp_blockchain::HeaderBackend, + sp_consensus::SyncOracle, + sp_consensus_aura::{Slot, SlotDuration}, + sp_core::crypto::Pair, + sp_inherents::CreateInherentDataProviders, + sp_keystore::KeystorePtr, + sp_runtime::traits::{Block as BlockT, Header as HeaderT, Member}, + std::{convert::TryFrom, error::Error, sync::Arc, time::Duration}, + tokio::sync::watch::Receiver, }; /// Parameters for [`run`].