diff --git a/cumulus/parachain-template/node/src/command.rs b/cumulus/parachain-template/node/src/command.rs
index ee497b413f5eae2d9c8ac424d074a36ecf439fa6..837cd15abf598bfaba48ec4128fde1234c6df2dc 100644
--- a/cumulus/parachain-template/node/src/command.rs
+++ b/cumulus/parachain-template/node/src/command.rs
@@ -5,7 +5,7 @@ use cumulus_client_cli::generate_genesis_block;
 use cumulus_primitives_core::ParaId;
 use frame_benchmarking_cli::{BenchmarkCmd, SUBSTRATE_REFERENCE_HARDWARE};
 use log::{info, warn};
-use parachain_template_runtime::{Block, RuntimeApi};
+use parachain_template_runtime::Block;
 use sc_cli::{
 	ChainSpec, CliConfiguration, DefaultConfigurationValues, ImportParams, KeystoreParams,
 	NetworkParams, Result, RuntimeVersion, SharedParams, SubstrateCli,
@@ -20,7 +20,7 @@ use sp_runtime::traits::{AccountIdConversion, Block as BlockT};
 use crate::{
 	chain_spec,
 	cli::{Cli, RelayChainCli, Subcommand},
-	service::{new_partial, TemplateRuntimeExecutor},
+	service::{new_partial, ParachainNativeExecutor},
 };
 
 fn load_spec(id: &str) -> std::result::Result<Box<dyn ChainSpec>, String> {
@@ -116,14 +116,7 @@ macro_rules! construct_async_run {
 	(|$components:ident, $cli:ident, $cmd:ident, $config:ident| $( $code:tt )* ) => {{
 		let runner = $cli.create_runner($cmd)?;
 		runner.async_run(|$config| {
-			let $components = new_partial::<
-				RuntimeApi,
-				TemplateRuntimeExecutor,
-				_
-			>(
-				&$config,
-				crate::service::parachain_build_import_queue,
-			)?;
+			let $components = new_partial(&$config)?;
 			let task_manager = $components.task_manager;
 			{ $( $code )* }.map(|v| (v, task_manager))
 		})
@@ -204,17 +197,14 @@ pub fn run() -> Result<()> {
 			match cmd {
 				BenchmarkCmd::Pallet(cmd) =>
 					if cfg!(feature = "runtime-benchmarks") {
-						runner.sync_run(|config| cmd.run::<Block, TemplateRuntimeExecutor>(config))
+						runner.sync_run(|config| cmd.run::<Block, ParachainNativeExecutor>(config))
 					} else {
 						Err("Benchmarking wasn't enabled when building the node. \
 					You can enable it with `--features runtime-benchmarks`."
 							.into())
 					},
 				BenchmarkCmd::Block(cmd) => runner.sync_run(|config| {
-					let partials = new_partial::<RuntimeApi, TemplateRuntimeExecutor, _>(
-						&config,
-						crate::service::parachain_build_import_queue,
-					)?;
+					let partials = new_partial(&config)?;
 					cmd.run(partials.client)
 				}),
 				#[cfg(not(feature = "runtime-benchmarks"))]
@@ -227,10 +217,7 @@ pub fn run() -> Result<()> {
 					.into()),
 				#[cfg(feature = "runtime-benchmarks")]
 				BenchmarkCmd::Storage(cmd) => runner.sync_run(|config| {
-					let partials = new_partial::<RuntimeApi, TemplateRuntimeExecutor, _>(
-						&config,
-						crate::service::parachain_build_import_queue,
-					)?;
+					let partials = new_partial(&config)?;
 					let db = partials.backend.expose_db();
 					let storage = partials.backend.expose_storage();
 
@@ -255,7 +242,7 @@ pub fn run() -> Result<()> {
 						.map_err(|e| format!("Error: {:?}", e))?;
 
 				runner.async_run(|config| {
-					Ok((cmd.run::<Block, TemplateRuntimeExecutor>(config), task_manager))
+					Ok((cmd.run::<Block, ParachainNativeExecutor>(config), task_manager))
 				})
 			} else {
 				Err("Try-runtime must be enabled by `--features try-runtime`.".into())
diff --git a/cumulus/parachain-template/node/src/service.rs b/cumulus/parachain-template/node/src/service.rs
index 91d8d54244d0a0c3571a1eb74be6e3bc6189015a..0980bb2cf78b6764f992f001446fcdd1b1e72f61 100644
--- a/cumulus/parachain-template/node/src/service.rs
+++ b/cumulus/parachain-template/node/src/service.rs
@@ -3,14 +3,9 @@
 // std
 use std::{sync::Arc, time::Duration};
 
-// rpc
-use jsonrpsee::RpcModule;
-
 use cumulus_client_cli::CollatorOptions;
 // Local Runtime Types
-use parachain_template_runtime::{
-	opaque::Block, AccountId, Balance, Hash, Index as Nonce, RuntimeApi,
-};
+use parachain_template_runtime::{opaque::Block, Hash, RuntimeApi};
 
 // Cumulus Imports
 use cumulus_client_consensus_aura::{AuraConsensus, BuildAuraConsensusParams, SlotProportion};
@@ -30,17 +25,15 @@ use sc_network::NetworkService;
 use sc_network_common::service::NetworkBlock;
 use sc_service::{Configuration, PartialComponents, TFullBackend, TFullClient, TaskManager};
 use sc_telemetry::{Telemetry, TelemetryHandle, TelemetryWorker, TelemetryWorkerHandle};
-use sp_api::ConstructRuntimeApi;
 use sp_keystore::SyncCryptoStorePtr;
-use sp_runtime::traits::BlakeTwo256;
 use substrate_prometheus_endpoint::Registry;
 
 use polkadot_service::CollatorPair;
 
-/// Native executor instance.
-pub struct TemplateRuntimeExecutor;
+/// Native executor type.
+pub struct ParachainNativeExecutor;
 
-impl sc_executor::NativeExecutionDispatch for TemplateRuntimeExecutor {
+impl sc_executor::NativeExecutionDispatch for ParachainNativeExecutor {
 	type ExtendHostFunctions = frame_benchmarking::benchmarking::HostFunctions;
 
 	fn dispatch(method: &str, data: &[u8]) -> Option<Vec<u8>> {
@@ -52,59 +45,29 @@ impl sc_executor::NativeExecutionDispatch for TemplateRuntimeExecutor {
 	}
 }
 
+type ParachainExecutor = NativeElseWasmExecutor<ParachainNativeExecutor>;
+
+type ParachainClient = TFullClient<Block, RuntimeApi, ParachainExecutor>;
+
+type ParachainBackend = TFullBackend<Block>;
+
 /// Starts a `ServiceBuilder` for a full service.
 ///
 /// Use this macro if you don't actually need the full service, but just the builder in order to
 /// be able to perform chain operations.
-#[allow(clippy::type_complexity)]
-pub fn new_partial<RuntimeApi, Executor, BIQ>(
+pub fn new_partial(
 	config: &Configuration,
-	build_import_queue: BIQ,
 ) -> Result<
 	PartialComponents<
-		TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>,
-		TFullBackend<Block>,
+		ParachainClient,
+		ParachainBackend,
 		(),
-		sc_consensus::DefaultImportQueue<
-			Block,
-			TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>,
-		>,
-		sc_transaction_pool::FullPool<
-			Block,
-			TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>,
-		>,
+		sc_consensus::DefaultImportQueue<Block, ParachainClient>,
+		sc_transaction_pool::FullPool<Block, ParachainClient>,
 		(Option<Telemetry>, Option<TelemetryWorkerHandle>),
 	>,
 	sc_service::Error,
->
-where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>>
-		+ Send
-		+ Sync
-		+ 'static,
-	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
-		+ sp_api::Metadata<Block>
-		+ sp_session::SessionKeys<Block>
-		+ sp_api::ApiExt<
-			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
-		> + sp_offchain::OffchainWorkerApi<Block>
-		+ sp_block_builder::BlockBuilder<Block>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
-	Executor: sc_executor::NativeExecutionDispatch + 'static,
-	BIQ: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>>,
-		&Configuration,
-		Option<TelemetryHandle>,
-		&TaskManager,
-	) -> Result<
-		sc_consensus::DefaultImportQueue<
-			Block,
-			TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>,
-		>,
-		sc_service::Error,
-	>,
-{
+> {
 	let telemetry = config
 		.telemetry_endpoints
 		.clone()
@@ -116,7 +79,7 @@ where
 		})
 		.transpose()?;
 
-	let executor = sc_executor::NativeElseWasmExecutor::<Executor>::new(
+	let executor = ParachainExecutor::new(
 		config.wasm_method,
 		config.default_heap_pages,
 		config.max_runtime_instances,
@@ -153,7 +116,7 @@ where
 		&task_manager,
 	)?;
 
-	let params = PartialComponents {
+	Ok(PartialComponents {
 		backend,
 		client,
 		import_queue,
@@ -162,9 +125,7 @@ where
 		transaction_pool,
 		select_chain: (),
 		other: (telemetry, telemetry_worker_handle),
-	};
-
-	Ok(params)
+	})
 }
 
 async fn build_relay_chain_interface(
@@ -192,74 +153,16 @@ async fn build_relay_chain_interface(
 ///
 /// This is the actual implementation that is abstract over the executor and the runtime api.
 #[sc_tracing::logging::prefix_logs_with("Parachain")]
-async fn start_node_impl<RuntimeApi, Executor, RB, BIQ, BIC>(
+async fn start_node_impl(
 	parachain_config: Configuration,
 	polkadot_config: Configuration,
 	collator_options: CollatorOptions,
 	id: ParaId,
-	_rpc_ext_builder: RB,
-	build_import_queue: BIQ,
-	build_consensus: BIC,
 	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>>,
-)>
-where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>>
-		+ Send
-		+ Sync
-		+ 'static,
-	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
-		+ sp_api::Metadata<Block>
-		+ sp_session::SessionKeys<Block>
-		+ sp_api::ApiExt<
-			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
-		> + sp_offchain::OffchainWorkerApi<Block>
-		+ sp_block_builder::BlockBuilder<Block>
-		+ cumulus_primitives_core::CollectCollationInfo<Block>
-		+ pallet_transaction_payment_rpc::TransactionPaymentRuntimeApi<Block, Balance>
-		+ substrate_frame_rpc_system::AccountNonceApi<Block, AccountId, Nonce>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
-	Executor: sc_executor::NativeExecutionDispatch + 'static,
-	RB: Fn(
-			Arc<TFullClient<Block, RuntimeApi, Executor>>,
-		) -> Result<RpcModule<()>, sc_service::Error>
-		+ Send
-		+ 'static,
-	BIQ: FnOnce(
-			Arc<TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>>,
-			&Configuration,
-			Option<TelemetryHandle>,
-			&TaskManager,
-		) -> Result<
-			sc_consensus::DefaultImportQueue<
-				Block,
-				TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>,
-			>,
-			sc_service::Error,
-		> + 'static,
-	BIC: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>>,
-		Option<&Registry>,
-		Option<TelemetryHandle>,
-		&TaskManager,
-		Arc<dyn RelayChainInterface>,
-		Arc<
-			sc_transaction_pool::FullPool<
-				Block,
-				TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<Executor>>,
-			>,
-		>,
-		Arc<NetworkService<Block, Hash>>,
-		SyncCryptoStorePtr,
-		bool,
-	) -> Result<Box<dyn ParachainConsensus<Block>>, sc_service::Error>,
-{
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient>)> {
 	let parachain_config = prepare_node_config(parachain_config);
 
-	let params = new_partial::<RuntimeApi, Executor, BIQ>(&parachain_config, build_import_queue)?;
+	let params = new_partial(&parachain_config)?;
 	let (mut telemetry, telemetry_worker_handle) = params.other;
 
 	let client = params.client.clone();
@@ -360,6 +263,7 @@ where
 			network,
 			params.keystore_container.sync_keystore(),
 			force_authoring,
+			id,
 		)?;
 
 		let spawner = task_manager.spawn_handle();
@@ -398,19 +302,12 @@ where
 }
 
 /// Build the import queue for the parachain runtime.
-#[allow(clippy::type_complexity)]
-pub fn parachain_build_import_queue(
-	client: Arc<TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<TemplateRuntimeExecutor>>>,
+fn build_import_queue(
+	client: Arc<ParachainClient>,
 	config: &Configuration,
 	telemetry: Option<TelemetryHandle>,
 	task_manager: &TaskManager,
-) -> Result<
-	sc_consensus::DefaultImportQueue<
-		Block,
-		TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<TemplateRuntimeExecutor>>,
-	>,
-	sc_service::Error,
-> {
+) -> Result<sc_consensus::DefaultImportQueue<Block, ParachainClient>, sc_service::Error> {
 	let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client)?;
 
 	cumulus_client_consensus_aura::import_queue::<
@@ -441,88 +338,81 @@ pub fn parachain_build_import_queue(
 	.map_err(Into::into)
 }
 
-/// Start a parachain node.
-pub async fn start_parachain_node(
-	parachain_config: Configuration,
-	polkadot_config: Configuration,
-	collator_options: CollatorOptions,
+fn build_consensus(
+	client: Arc<ParachainClient>,
+	prometheus_registry: Option<&Registry>,
+	telemetry: Option<TelemetryHandle>,
+	task_manager: &TaskManager,
+	relay_chain_interface: Arc<dyn RelayChainInterface>,
+	transaction_pool: Arc<sc_transaction_pool::FullPool<Block, ParachainClient>>,
+	sync_oracle: Arc<NetworkService<Block, Hash>>,
+	keystore: SyncCryptoStorePtr,
+	force_authoring: bool,
 	id: ParaId,
-	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, NativeElseWasmExecutor<TemplateRuntimeExecutor>>>,
-)> {
-	start_node_impl::<RuntimeApi, TemplateRuntimeExecutor, _, _, _>(
-		parachain_config,
-		polkadot_config,
-		collator_options,
-		id,
-		|_| Ok(RpcModule::new(())),
-		parachain_build_import_queue,
-		|client,
-		 prometheus_registry,
-		 telemetry,
-		 task_manager,
-		 relay_chain_interface,
-		 transaction_pool,
-		 sync_oracle,
-		 keystore,
-		 force_authoring| {
-			let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client)?;
-
-			let proposer_factory = sc_basic_authorship::ProposerFactory::with_proof_recording(
-				task_manager.spawn_handle(),
-				client.clone(),
-				transaction_pool,
-				prometheus_registry,
-				telemetry.clone(),
-			);
+) -> Result<Box<dyn ParachainConsensus<Block>>, sc_service::Error> {
+	let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client)?;
+
+	let proposer_factory = sc_basic_authorship::ProposerFactory::with_proof_recording(
+		task_manager.spawn_handle(),
+		client.clone(),
+		transaction_pool,
+		prometheus_registry,
+		telemetry.clone(),
+	);
 
-			Ok(AuraConsensus::build::<sp_consensus_aura::sr25519::AuthorityPair, _, _, _, _, _, _>(
-				BuildAuraConsensusParams {
-					proposer_factory,
-					create_inherent_data_providers: move |_, (relay_parent, validation_data)| {
-						let relay_chain_interface = relay_chain_interface.clone();
-						async move {
-							let parachain_inherent =
-							cumulus_primitives_parachain_inherent::ParachainInherentData::create_at(
-								relay_parent,
-								&relay_chain_interface,
-								&validation_data,
-								id,
-							).await;
-							let timestamp = sp_timestamp::InherentDataProvider::from_system_time();
-
-							let slot =
+	let params = BuildAuraConsensusParams {
+		proposer_factory,
+		create_inherent_data_providers: move |_, (relay_parent, validation_data)| {
+			let relay_chain_interface = relay_chain_interface.clone();
+			async move {
+				let parachain_inherent =
+					cumulus_primitives_parachain_inherent::ParachainInherentData::create_at(
+						relay_parent,
+						&relay_chain_interface,
+						&validation_data,
+						id,
+					)
+					.await;
+				let timestamp = sp_timestamp::InherentDataProvider::from_system_time();
+
+				let slot =
 						sp_consensus_aura::inherents::InherentDataProvider::from_timestamp_and_slot_duration(
 							*timestamp,
 							slot_duration,
 						);
 
-							let parachain_inherent = parachain_inherent.ok_or_else(|| {
-								Box::<dyn std::error::Error + Send + Sync>::from(
-									"Failed to create parachain inherent",
-								)
-							})?;
-							Ok((slot, timestamp, parachain_inherent))
-						}
-					},
-					block_import: client.clone(),
-					para_client: client,
-					backoff_authoring_blocks: Option::<()>::None,
-					sync_oracle,
-					keystore,
-					force_authoring,
-					slot_duration,
-					// We got around 500ms for proposing
-					block_proposal_slot_portion: SlotProportion::new(1f32 / 24f32),
-					// And a maximum of 750ms if slots are skipped
-					max_block_proposal_slot_portion: Some(SlotProportion::new(1f32 / 16f32)),
-					telemetry,
-				},
-			))
+				let parachain_inherent = parachain_inherent.ok_or_else(|| {
+					Box::<dyn std::error::Error + Send + Sync>::from(
+						"Failed to create parachain inherent",
+					)
+				})?;
+				Ok((slot, timestamp, parachain_inherent))
+			}
 		},
-		hwbench,
-	)
-	.await
+		block_import: client.clone(),
+		para_client: client,
+		backoff_authoring_blocks: Option::<()>::None,
+		sync_oracle,
+		keystore,
+		force_authoring,
+		slot_duration,
+		// We got around 500ms for proposing
+		block_proposal_slot_portion: SlotProportion::new(1f32 / 24f32),
+		// And a maximum of 750ms if slots are skipped
+		max_block_proposal_slot_portion: Some(SlotProportion::new(1f32 / 16f32)),
+		telemetry,
+	};
+
+	Ok(AuraConsensus::build::<sp_consensus_aura::sr25519::AuthorityPair, _, _, _, _, _, _>(params))
+}
+
+/// Start a parachain node.
+pub async fn start_parachain_node(
+	parachain_config: Configuration,
+	polkadot_config: Configuration,
+	collator_options: CollatorOptions,
+	id: ParaId,
+	hwbench: Option<sc_sysinfo::HwBench>,
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient>)> {
+	start_node_impl(parachain_config, polkadot_config, collator_options, id, hwbench).await
 }
diff --git a/cumulus/polkadot-parachain/src/service.rs b/cumulus/polkadot-parachain/src/service.rs
index ca47af08eece3bd96cd0292343e57ef97b814bf8..2f38da0bd5d0d72315531779a6f235ec988c8318 100644
--- a/cumulus/polkadot-parachain/src/service.rs
+++ b/cumulus/polkadot-parachain/src/service.rs
@@ -69,6 +69,10 @@ type HostFunctions = sp_io::SubstrateHostFunctions;
 type HostFunctions =
 	(sp_io::SubstrateHostFunctions, frame_benchmarking::benchmarking::HostFunctions);
 
+type ParachainClient<RuntimeApi> = TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>;
+
+type ParachainBackend = TFullBackend<Block>;
+
 /// Native executor instance.
 pub struct ShellRuntimeExecutor;
 
@@ -153,45 +157,33 @@ pub fn new_partial<RuntimeApi, BIQ>(
 	build_import_queue: BIQ,
 ) -> Result<
 	PartialComponents<
-		TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-		TFullBackend<Block>,
+		ParachainClient<RuntimeApi>,
+		ParachainBackend,
 		(),
-		sc_consensus::DefaultImportQueue<
-			Block,
-			TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-		>,
-		sc_transaction_pool::FullPool<
-			Block,
-			TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-		>,
+		sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>,
+		sc_transaction_pool::FullPool<Block, ParachainClient<RuntimeApi>>,
 		(Option<Telemetry>, Option<TelemetryWorkerHandle>),
 	>,
 	sc_service::Error,
 >
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
 	BIQ: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+		Arc<ParachainClient<RuntimeApi>>,
 		&Configuration,
 		Option<TelemetryHandle>,
 		&TaskManager,
 	) -> Result<
-		sc_consensus::DefaultImportQueue<
-			Block,
-			TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-		>,
+		sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>,
 		sc_service::Error,
 	>,
 {
@@ -292,54 +284,38 @@ async fn start_shell_node_impl<RuntimeApi, RB, BIQ, BIC>(
 	build_import_queue: BIQ,
 	build_consensus: BIC,
 	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-)>
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient<RuntimeApi>>)>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>
 		+ cumulus_primitives_core::CollectCollationInfo<Block>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
-	RB: Fn(
-			Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-		) -> Result<jsonrpsee::RpcModule<()>, sc_service::Error>
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
+	RB: Fn(Arc<ParachainClient<RuntimeApi>>) -> Result<jsonrpsee::RpcModule<()>, sc_service::Error>
 		+ Send
 		+ 'static,
 	BIQ: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+		Arc<ParachainClient<RuntimeApi>>,
 		&Configuration,
 		Option<TelemetryHandle>,
 		&TaskManager,
 	) -> Result<
-		sc_consensus::DefaultImportQueue<
-			Block,
-			TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-		>,
+		sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>,
 		sc_service::Error,
 	>,
 	BIC: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+		Arc<ParachainClient<RuntimeApi>>,
 		Option<&Registry>,
 		Option<TelemetryHandle>,
 		&TaskManager,
 		Arc<dyn RelayChainInterface>,
-		Arc<
-			sc_transaction_pool::FullPool<
-				Block,
-				TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-			>,
-		>,
+		Arc<sc_transaction_pool::FullPool<Block, ParachainClient<RuntimeApi>>>,
 		Arc<NetworkService<Block, Hash>>,
 		SyncCryptoStorePtr,
 		bool,
@@ -488,56 +464,40 @@ async fn start_node_impl<RuntimeApi, RB, BIQ, BIC>(
 	build_import_queue: BIQ,
 	build_consensus: BIC,
 	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-)>
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient<RuntimeApi>>)>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>
 		+ cumulus_primitives_core::CollectCollationInfo<Block>
 		+ pallet_transaction_payment_rpc::TransactionPaymentRuntimeApi<Block, Balance>
 		+ frame_rpc_system::AccountNonceApi<Block, AccountId, Nonce>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
-	RB: Fn(
-			Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-		) -> Result<jsonrpsee::RpcModule<()>, sc_service::Error>
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
+	RB: Fn(Arc<ParachainClient<RuntimeApi>>) -> Result<jsonrpsee::RpcModule<()>, sc_service::Error>
 		+ Send
 		+ 'static,
 	BIQ: FnOnce(
-			Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+			Arc<ParachainClient<RuntimeApi>>,
 			&Configuration,
 			Option<TelemetryHandle>,
 			&TaskManager,
 		) -> Result<
-			sc_consensus::DefaultImportQueue<
-				Block,
-				TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-			>,
+			sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>,
 			sc_service::Error,
 		> + 'static,
 	BIC: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+		Arc<ParachainClient<RuntimeApi>>,
 		Option<&Registry>,
 		Option<TelemetryHandle>,
 		&TaskManager,
 		Arc<dyn RelayChainInterface>,
-		Arc<
-			sc_transaction_pool::FullPool<
-				Block,
-				TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-			>,
-		>,
+		Arc<sc_transaction_pool::FullPool<Block, ParachainClient<RuntimeApi>>>,
 		Arc<NetworkService<Block, Hash>>,
 		SyncCryptoStorePtr,
 		bool,
@@ -686,17 +646,12 @@ where
 
 /// Build the import queue for the rococo parachain runtime.
 pub fn rococo_parachain_build_import_queue(
-	client: Arc<
-		TFullClient<Block, rococo_parachain_runtime::RuntimeApi, WasmExecutor<HostFunctions>>,
-	>,
+	client: Arc<ParachainClient<rococo_parachain_runtime::RuntimeApi>>,
 	config: &Configuration,
 	telemetry: Option<TelemetryHandle>,
 	task_manager: &TaskManager,
 ) -> Result<
-	sc_consensus::DefaultImportQueue<
-		Block,
-		TFullClient<Block, rococo_parachain_runtime::RuntimeApi, WasmExecutor<HostFunctions>>,
-	>,
+	sc_consensus::DefaultImportQueue<Block, ParachainClient<rococo_parachain_runtime::RuntimeApi>>,
 	sc_service::Error,
 > {
 	let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client)?;
@@ -738,7 +693,7 @@ pub async fn start_rococo_parachain_node(
 	hwbench: Option<sc_sysinfo::HwBench>,
 ) -> sc_service::error::Result<(
 	TaskManager,
-	Arc<TFullClient<Block, rococo_parachain_runtime::RuntimeApi, WasmExecutor<HostFunctions>>>,
+	Arc<ParachainClient<rococo_parachain_runtime::RuntimeApi>>,
 )> {
 	start_node_impl::<rococo_parachain_runtime::RuntimeApi, _, _, _>(
 		parachain_config,
@@ -820,31 +775,22 @@ pub async fn start_rococo_parachain_node(
 
 /// Build the import queue for the shell runtime.
 pub fn shell_build_import_queue<RuntimeApi>(
-	client: Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+	client: Arc<ParachainClient<RuntimeApi>>,
 	config: &Configuration,
 	_: Option<TelemetryHandle>,
 	task_manager: &TaskManager,
-) -> Result<
-	sc_consensus::DefaultImportQueue<
-		Block,
-		TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-	>,
-	sc_service::Error,
->
+) -> Result<sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>, sc_service::Error>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
 {
 	cumulus_client_consensus_relay_chain::import_queue(
 		client.clone(),
@@ -863,25 +809,19 @@ pub async fn start_shell_node<RuntimeApi>(
 	collator_options: CollatorOptions,
 	id: ParaId,
 	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-)>
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient<RuntimeApi>>)>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>
 		+ cumulus_primitives_core::CollectCollationInfo<Block>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
 {
 	start_shell_node_impl::<RuntimeApi, _, _, _>(
 		parachain_config,
@@ -1048,32 +988,23 @@ where
 
 /// Build the import queue for Statemint and other Aura-based runtimes.
 pub fn aura_build_import_queue<RuntimeApi, AuraId: AppKey>(
-	client: Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+	client: Arc<ParachainClient<RuntimeApi>>,
 	config: &Configuration,
 	telemetry_handle: Option<TelemetryHandle>,
 	task_manager: &TaskManager,
-) -> Result<
-	sc_consensus::DefaultImportQueue<
-		Block,
-		TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-	>,
-	sc_service::Error,
->
+) -> Result<sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>, sc_service::Error>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>
 		+ sp_consensus_aura::AuraApi<Block, <<AuraId as AppKey>::Pair as Pair>::Public>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
 	<<AuraId as AppKey>::Pair as Pair>::Signature:
 		TryFrom<Vec<u8>> + std::hash::Hash + sp_runtime::traits::Member + Codec,
 {
@@ -1131,28 +1062,22 @@ pub async fn start_generic_aura_node<RuntimeApi, AuraId: AppKey>(
 	collator_options: CollatorOptions,
 	id: ParaId,
 	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-)>
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient<RuntimeApi>>)>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>
 		+ cumulus_primitives_core::CollectCollationInfo<Block>
 		+ sp_consensus_aura::AuraApi<Block, <<AuraId as AppKey>::Pair as Pair>::Public>
 		+ pallet_transaction_payment_rpc::TransactionPaymentRuntimeApi<Block, Balance>
 		+ frame_rpc_system::AccountNonceApi<Block, AccountId, Nonce>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
 	<<AuraId as AppKey>::Pair as Pair>::Signature:
 		TryFrom<Vec<u8>> + std::hash::Hash + sp_runtime::traits::Member + Codec,
 {
@@ -1302,56 +1227,40 @@ async fn start_contracts_rococo_node_impl<RuntimeApi, RB, BIQ, BIC>(
 	build_import_queue: BIQ,
 	build_consensus: BIC,
 	hwbench: Option<sc_sysinfo::HwBench>,
-) -> sc_service::error::Result<(
-	TaskManager,
-	Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-)>
+) -> sc_service::error::Result<(TaskManager, Arc<ParachainClient<RuntimeApi>>)>
 where
-	RuntimeApi: ConstructRuntimeApi<Block, TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>
-		+ Send
-		+ Sync
-		+ 'static,
+	RuntimeApi: ConstructRuntimeApi<Block, ParachainClient<RuntimeApi>> + Send + Sync + 'static,
 	RuntimeApi::RuntimeApi: sp_transaction_pool::runtime_api::TaggedTransactionQueue<Block>
 		+ sp_api::Metadata<Block>
 		+ sp_session::SessionKeys<Block>
 		+ sp_api::ApiExt<
 			Block,
-			StateBackend = sc_client_api::StateBackendFor<TFullBackend<Block>, Block>,
+			StateBackend = sc_client_api::StateBackendFor<ParachainBackend, Block>,
 		> + sp_offchain::OffchainWorkerApi<Block>
 		+ sp_block_builder::BlockBuilder<Block>
 		+ cumulus_primitives_core::CollectCollationInfo<Block>
 		+ pallet_transaction_payment_rpc::TransactionPaymentRuntimeApi<Block, Balance>
 		+ frame_rpc_system::AccountNonceApi<Block, AccountId, Nonce>,
-	sc_client_api::StateBackendFor<TFullBackend<Block>, Block>: sp_api::StateBackend<BlakeTwo256>,
-	RB: Fn(
-			Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
-		) -> Result<jsonrpsee::RpcModule<()>, sc_service::Error>
+	sc_client_api::StateBackendFor<ParachainBackend, Block>: sp_api::StateBackend<BlakeTwo256>,
+	RB: Fn(Arc<ParachainClient<RuntimeApi>>) -> Result<jsonrpsee::RpcModule<()>, sc_service::Error>
 		+ Send
 		+ 'static,
 	BIQ: FnOnce(
-			Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+			Arc<ParachainClient<RuntimeApi>>,
 			&Configuration,
 			Option<TelemetryHandle>,
 			&TaskManager,
 		) -> Result<
-			sc_consensus::DefaultImportQueue<
-				Block,
-				TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-			>,
+			sc_consensus::DefaultImportQueue<Block, ParachainClient<RuntimeApi>>,
 			sc_service::Error,
 		> + 'static,
 	BIC: FnOnce(
-		Arc<TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>>,
+		Arc<ParachainClient<RuntimeApi>>,
 		Option<&Registry>,
 		Option<TelemetryHandle>,
 		&TaskManager,
 		Arc<dyn RelayChainInterface>,
-		Arc<
-			sc_transaction_pool::FullPool<
-				Block,
-				TFullClient<Block, RuntimeApi, WasmExecutor<HostFunctions>>,
-			>,
-		>,
+		Arc<sc_transaction_pool::FullPool<Block, ParachainClient<RuntimeApi>>>,
 		Arc<NetworkService<Block, Hash>>,
 		SyncCryptoStorePtr,
 		bool,
@@ -1500,17 +1409,12 @@ where
 
 #[allow(clippy::type_complexity)]
 pub fn contracts_rococo_build_import_queue(
-	client: Arc<
-		TFullClient<Block, contracts_rococo_runtime::RuntimeApi, WasmExecutor<HostFunctions>>,
-	>,
+	client: Arc<ParachainClient<contracts_rococo_runtime::RuntimeApi>>,
 	config: &Configuration,
 	telemetry: Option<TelemetryHandle>,
 	task_manager: &TaskManager,
 ) -> Result<
-	sc_consensus::DefaultImportQueue<
-		Block,
-		TFullClient<Block, contracts_rococo_runtime::RuntimeApi, WasmExecutor<HostFunctions>>,
-	>,
+	sc_consensus::DefaultImportQueue<Block, ParachainClient<contracts_rococo_runtime::RuntimeApi>>,
 	sc_service::Error,
 > {
 	let slot_duration = cumulus_client_consensus_aura::slot_duration(&*client)?;
@@ -1552,7 +1456,7 @@ pub async fn start_contracts_rococo_node(
 	hwbench: Option<sc_sysinfo::HwBench>,
 ) -> sc_service::error::Result<(
 	TaskManager,
-	Arc<TFullClient<Block, contracts_rococo_runtime::RuntimeApi, WasmExecutor<HostFunctions>>>,
+	Arc<ParachainClient<contracts_rococo_runtime::RuntimeApi>>,
 )> {
 	start_contracts_rococo_node_impl::<contracts_rococo_runtime::RuntimeApi, _, _, _>(
 		parachain_config,