From 8962e04029658ab467a3a102cc1a4295a7e7255d Mon Sep 17 00:00:00 2001 From: Robert Habermeier Date: Mon, 4 Oct 2021 16:53:36 +0200 Subject: [PATCH 1/3] Make candidate validation timeouts configurable (#4001) * pvf: make execution timeout configurable * guide: add timeouts to candidate validation params * add timeouts to candidate validation messages * fmt * port backing to use the backing pvf timeout * port approval-voting to use the execution timeout * port dispute participation to use the correct timeout * fmt * address grumbles & test failure --- node/core/approval-voting/src/lib.rs | 3 +- node/core/backing/src/lib.rs | 8 +- node/core/backing/src/tests.rs | 28 +++-- node/core/candidate-validation/src/lib.rs | 14 ++- node/core/candidate-validation/src/tests.rs | 8 ++ node/core/dispute-participation/src/lib.rs | 7 +- node/core/dispute-participation/src/tests.rs | 16 +-- node/core/pvf/src/execute/queue.rs | 20 +++- node/core/pvf/src/execute/worker.rs | 5 +- node/core/pvf/src/host.rs | 100 +++++++++++++----- node/core/pvf/tests/it/main.rs | 3 + node/overseer/examples/minimal-example.rs | 1 + node/overseer/src/tests.rs | 8 +- node/primitives/src/lib.rs | 13 ++- node/subsystem-types/src/messages.rs | 9 +- .../src/node/approval/approval-voting.md | 2 +- .../src/node/backing/candidate-backing.md | 2 +- .../node/disputes/dispute-participation.md | 2 +- .../src/types/overseer-protocol.md | 5 + 19 files changed, 192 insertions(+), 62 deletions(-) diff --git a/node/core/approval-voting/src/lib.rs b/node/core/approval-voting/src/lib.rs index 985cc9d6c541..af46ad55fa04 100644 --- a/node/core/approval-voting/src/lib.rs +++ b/node/core/approval-voting/src/lib.rs @@ -27,7 +27,7 @@ use polkadot_node_primitives::{ approval::{ BlockApprovalMeta, DelayTranche, IndirectAssignmentCert, IndirectSignedApprovalVote, }, - SignedDisputeStatement, ValidationResult, + SignedDisputeStatement, ValidationResult, APPROVAL_EXECUTION_TIMEOUT, }; use polkadot_node_subsystem::{ errors::RecoveryError, @@ -2235,6 +2235,7 @@ async fn launch_approval( validation_code, candidate.descriptor.clone(), available_data.pov, + APPROVAL_EXECUTION_TIMEOUT, val_tx, ) .into(), diff --git a/node/core/backing/src/lib.rs b/node/core/backing/src/lib.rs index bd24244a01f8..38fab2d791c2 100644 --- a/node/core/backing/src/lib.rs +++ b/node/core/backing/src/lib.rs @@ -32,6 +32,7 @@ use futures::{ use polkadot_node_primitives::{ AvailableData, PoV, SignedDisputeStatement, SignedFullStatement, Statement, ValidationResult, + BACKING_EXECUTION_TIMEOUT, }; use polkadot_node_subsystem_util::{ self as util, @@ -380,7 +381,12 @@ async fn request_candidate_validation( let (tx, rx) = oneshot::channel(); sender - .send_message(CandidateValidationMessage::ValidateFromChainState(candidate, pov, tx)) + .send_message(CandidateValidationMessage::ValidateFromChainState( + candidate, + pov, + BACKING_EXECUTION_TIMEOUT, + tx, + )) .await; match rx.await { diff --git a/node/core/backing/src/tests.rs b/node/core/backing/src/tests.rs index 68c1e30dfd31..a262dd45d470 100644 --- a/node/core/backing/src/tests.rs +++ b/node/core/backing/src/tests.rs @@ -317,9 +317,10 @@ fn backing_second_works() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate.descriptor() => { + ) if pov == pov && &c == candidate.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Ok( ValidationResult::Valid(CandidateCommitments { head_data: expected_head_data.clone(), @@ -476,9 +477,10 @@ fn backing_works() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate_a.descriptor() => { + ) if pov == pov && &c == candidate_a.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Ok( ValidationResult::Valid(CandidateCommitments { head_data: expected_head_data.clone(), @@ -669,9 +671,10 @@ fn backing_works_while_validation_ongoing() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate_a.descriptor() => { + ) if pov == pov && &c == candidate_a.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { // we never validate the candidate. our local node // shouldn't issue any statements. std::mem::forget(tx); @@ -834,9 +837,10 @@ fn backing_misbehavior_works() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate_a.descriptor() => { + ) if pov == pov && &c == candidate_a.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Ok( ValidationResult::Valid(CandidateCommitments { head_data: expected_head_data.clone(), @@ -980,9 +984,10 @@ fn backing_dont_second_invalid() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate_a.descriptor() => { + ) if pov == pov && &c == candidate_a.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Ok(ValidationResult::Invalid(InvalidCandidate::BadReturn))).unwrap(); } ); @@ -1008,9 +1013,10 @@ fn backing_dont_second_invalid() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate_b.descriptor() => { + ) if pov == pov && &c == candidate_b.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Ok( ValidationResult::Valid(CandidateCommitments { head_data: expected_head_data.clone(), @@ -1138,9 +1144,10 @@ fn backing_second_after_first_fails_works() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate.descriptor() => { + ) if pov == pov && &c == candidate.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Ok(ValidationResult::Invalid(InvalidCandidate::BadReturn))).unwrap(); } ); @@ -1186,6 +1193,7 @@ fn backing_second_after_first_fails_works() { _, pov, _, + _, ) ) => { assert_eq!(&*pov, &pov_to_second); @@ -1270,9 +1278,10 @@ fn backing_works_after_failed_validation() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, tx, ) - ) if pov == pov && &c == candidate.descriptor() => { + ) if pov == pov && &c == candidate.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT => { tx.send(Err(ValidationFailed("Internal test error".into()))).unwrap(); } ); @@ -1646,9 +1655,10 @@ fn retry_works() { CandidateValidationMessage::ValidateFromChainState( c, pov, + timeout, _tx, ) - ) if pov == pov && &c == candidate.descriptor() + ) if pov == pov && &c == candidate.descriptor() && timeout == BACKING_EXECUTION_TIMEOUT ); virtual_overseer }); diff --git a/node/core/candidate-validation/src/lib.rs b/node/core/candidate-validation/src/lib.rs index 9d2852efcc59..c9e78db77c47 100644 --- a/node/core/candidate-validation/src/lib.rs +++ b/node/core/candidate-validation/src/lib.rs @@ -48,7 +48,7 @@ use parity_scale_codec::Encode; use futures::{channel::oneshot, prelude::*}; -use std::{path::PathBuf, sync::Arc}; +use std::{path::PathBuf, sync::Arc, time::Duration}; use async_trait::async_trait; @@ -135,6 +135,7 @@ where CandidateValidationMessage::ValidateFromChainState( descriptor, pov, + timeout, response_sender, ) => { let bg = { @@ -149,6 +150,7 @@ where validation_host, descriptor, pov, + timeout, &metrics, ) .await; @@ -165,6 +167,7 @@ where validation_code, descriptor, pov, + timeout, response_sender, ) => { let bg = { @@ -179,6 +182,7 @@ where validation_code, descriptor, pov, + timeout, &metrics, ) .await; @@ -322,6 +326,7 @@ async fn validate_from_chain_state( validation_host: ValidationHost, descriptor: CandidateDescriptor, pov: Arc, + timeout: Duration, metrics: &Metrics, ) -> Result where @@ -347,6 +352,7 @@ where validation_code, descriptor.clone(), pov, + timeout, metrics, ) .await; @@ -377,6 +383,7 @@ async fn validate_candidate_exhaustive( validation_code: ValidationCode, descriptor: CandidateDescriptor, pov: Arc, + timeout: Duration, metrics: &Metrics, ) -> Result { let _timer = metrics.time_validate_candidate_exhaustive(); @@ -430,7 +437,7 @@ async fn validate_candidate_exhaustive( }; let result = validation_backend - .validate_candidate(raw_validation_code.to_vec(), params) + .validate_candidate(raw_validation_code.to_vec(), timeout, params) .await; if let Err(ref e) = result { @@ -475,6 +482,7 @@ trait ValidationBackend { async fn validate_candidate( &mut self, raw_validation_code: Vec, + timeout: Duration, params: ValidationParams, ) -> Result; } @@ -484,12 +492,14 @@ impl ValidationBackend for ValidationHost { async fn validate_candidate( &mut self, raw_validation_code: Vec, + timeout: Duration, params: ValidationParams, ) -> Result { let (tx, rx) = oneshot::channel(); if let Err(err) = self .execute_pvf( Pvf::from_code(raw_validation_code), + timeout, params.encode(), polkadot_node_core_pvf::Priority::Normal, tx, diff --git a/node/core/candidate-validation/src/tests.rs b/node/core/candidate-validation/src/tests.rs index f067ead6cd3c..30fa96ccc398 100644 --- a/node/core/candidate-validation/src/tests.rs +++ b/node/core/candidate-validation/src/tests.rs @@ -341,6 +341,7 @@ impl ValidationBackend for MockValidatorBackend { async fn validate_candidate( &mut self, _raw_validation_code: Vec, + _timeout: Duration, _params: ValidationParams, ) -> Result { self.result.clone() @@ -384,6 +385,7 @@ fn candidate_validation_ok_is_ok() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )) .unwrap(); @@ -426,6 +428,7 @@ fn candidate_validation_bad_return_is_invalid() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )) .unwrap(); @@ -461,6 +464,7 @@ fn candidate_validation_timeout_is_internal_error() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )); @@ -495,6 +499,7 @@ fn candidate_validation_code_mismatch_is_invalid() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )) .unwrap(); @@ -534,6 +539,7 @@ fn compressed_code_works() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )); @@ -573,6 +579,7 @@ fn code_decompression_failure_is_invalid() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )); @@ -613,6 +620,7 @@ fn pov_decompression_failure_is_invalid() { validation_code, descriptor, Arc::new(pov), + Duration::from_secs(0), &Default::default(), )); diff --git a/node/core/dispute-participation/src/lib.rs b/node/core/dispute-participation/src/lib.rs index 85b31f0a86f4..21258ad5be11 100644 --- a/node/core/dispute-participation/src/lib.rs +++ b/node/core/dispute-participation/src/lib.rs @@ -22,7 +22,7 @@ use futures::{channel::oneshot, prelude::*}; -use polkadot_node_primitives::ValidationResult; +use polkadot_node_primitives::{ValidationResult, APPROVAL_EXECUTION_TIMEOUT}; use polkadot_node_subsystem::{ errors::{RecoveryError, RuntimeApiError}, messages::{ @@ -269,11 +269,16 @@ async fn participate( // we issue a request to validate the candidate with the provided exhaustive // parameters + // + // We use the approval execution timeout because this is intended to + // be run outside of backing and therefore should be subject to the + // same level of leeway. ctx.send_message(CandidateValidationMessage::ValidateFromExhaustive( available_data.validation_data, validation_code, candidate_receipt.descriptor.clone(), available_data.pov, + APPROVAL_EXECUTION_TIMEOUT, validation_tx, )) .await; diff --git a/node/core/dispute-participation/src/tests.rs b/node/core/dispute-participation/src/tests.rs index e2c98af53b5e..513f673f81a2 100644 --- a/node/core/dispute-participation/src/tests.rs +++ b/node/core/dispute-participation/src/tests.rs @@ -295,8 +295,8 @@ fn cast_invalid_vote_if_validation_fails_or_is_invalid() { assert_matches!( virtual_overseer.recv().await, AllMessages::CandidateValidation( - CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, tx) - ) => { + CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, timeout, tx) + ) if timeout == APPROVAL_EXECUTION_TIMEOUT => { tx.send(Ok(ValidationResult::Invalid(InvalidCandidate::Timeout))).unwrap(); }, "overseer did not receive candidate validation message", @@ -331,8 +331,8 @@ fn cast_invalid_vote_if_validation_passes_but_commitments_dont_match() { assert_matches!( virtual_overseer.recv().await, AllMessages::CandidateValidation( - CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, tx) - ) => { + CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, timeout, tx) + ) if timeout == APPROVAL_EXECUTION_TIMEOUT => { let mut commitments = CandidateCommitments::default(); // this should lead to a commitments hash mismatch commitments.processed_downward_messages = 42; @@ -371,8 +371,8 @@ fn cast_valid_vote_if_validation_passes() { assert_matches!( virtual_overseer.recv().await, AllMessages::CandidateValidation( - CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, tx) - ) => { + CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, timeout, tx) + ) if timeout == APPROVAL_EXECUTION_TIMEOUT => { tx.send(Ok(ValidationResult::Valid(Default::default(), Default::default()))).unwrap(); }, "overseer did not receive candidate validation message", @@ -408,8 +408,8 @@ fn failure_to_store_available_data_does_not_preclude_participation() { assert_matches!( virtual_overseer.recv().await, AllMessages::CandidateValidation( - CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, tx) - ) => { + CandidateValidationMessage::ValidateFromExhaustive(_, _, _, _, timeout, tx) + ) if timeout == APPROVAL_EXECUTION_TIMEOUT => { tx.send(Err(ValidationFailed("fail".to_string()))).unwrap(); }, "overseer did not receive candidate validation message", diff --git a/node/core/pvf/src/execute/queue.rs b/node/core/pvf/src/execute/queue.rs index 09e848196820..9376d7d76f78 100644 --- a/node/core/pvf/src/execute/queue.rs +++ b/node/core/pvf/src/execute/queue.rs @@ -38,11 +38,17 @@ slotmap::new_key_type! { struct Worker; } #[derive(Debug)] pub enum ToQueue { - Enqueue { artifact: ArtifactPathId, params: Vec, result_tx: ResultSender }, + Enqueue { + artifact: ArtifactPathId, + execution_timeout: Duration, + params: Vec, + result_tx: ResultSender, + }, } struct ExecuteJob { artifact: ArtifactPathId, + execution_timeout: Duration, params: Vec, result_tx: ResultSender, } @@ -167,14 +173,14 @@ async fn purge_dead(metrics: &Metrics, workers: &mut Workers) { } fn handle_to_queue(queue: &mut Queue, to_queue: ToQueue) { - let ToQueue::Enqueue { artifact, params, result_tx } = to_queue; + let ToQueue::Enqueue { artifact, execution_timeout, params, result_tx } = to_queue; tracing::debug!( target: LOG_TARGET, validation_code_hash = ?artifact.id.code_hash, "enqueueing an artifact for execution", ); queue.metrics.execute_enqueued(); - let job = ExecuteJob { artifact, params, result_tx }; + let job = ExecuteJob { artifact, execution_timeout, params, result_tx }; if let Some(available) = queue.workers.find_available() { assign(queue, available, job); @@ -326,7 +332,13 @@ fn assign(queue: &mut Queue, worker: Worker, job: ExecuteJob) { queue.mux.push( async move { let _timer = execution_timer; - let outcome = super::worker::start_work(idle, job.artifact.clone(), job.params).await; + let outcome = super::worker::start_work( + idle, + job.artifact.clone(), + job.execution_timeout, + job.params, + ) + .await; QueueEvent::StartWork(worker, outcome, job.artifact.id, job.result_tx) } .boxed(), diff --git a/node/core/pvf/src/execute/worker.rs b/node/core/pvf/src/execute/worker.rs index 97fe5aec3dbf..86e892672d9e 100644 --- a/node/core/pvf/src/execute/worker.rs +++ b/node/core/pvf/src/execute/worker.rs @@ -34,8 +34,6 @@ use parity_scale_codec::{Decode, Encode}; use polkadot_parachain::primitives::ValidationResult; use std::time::{Duration, Instant}; -const EXECUTION_TIMEOUT: Duration = Duration::from_secs(3); - /// Spawns a new worker with the given program path that acts as the worker and the spawn timeout. /// /// The program should be able to handle ` execute-worker ` invocation. @@ -69,6 +67,7 @@ pub enum Outcome { pub async fn start_work( worker: IdleWorker, artifact: ArtifactPathId, + execution_timeout: Duration, validation_params: Vec, ) -> Outcome { let IdleWorker { mut stream, pid } = worker; @@ -108,7 +107,7 @@ pub async fn start_work( Ok(response) => response, } }, - _ = Delay::new(EXECUTION_TIMEOUT).fuse() => { + _ = Delay::new(execution_timeout).fuse() => { tracing::warn!( target: LOG_TARGET, worker_pid = %pid, diff --git a/node/core/pvf/src/host.rs b/node/core/pvf/src/host.rs index 89b230bc90d7..40c30ca65c21 100644 --- a/node/core/pvf/src/host.rs +++ b/node/core/pvf/src/host.rs @@ -48,8 +48,8 @@ pub struct ValidationHost { } impl ValidationHost { - /// Execute PVF with the given code, parameters and priority. The result of execution will be sent - /// to the provided result sender. + /// Execute PVF with the given code, execution timeout, parameters and priority. + /// The result of execution will be sent to the provided result sender. /// /// This is async to accommodate the fact a possibility of back-pressure. In the vast majority of /// situations this function should return immediately. @@ -58,12 +58,13 @@ impl ValidationHost { pub async fn execute_pvf( &mut self, pvf: Pvf, + execution_timeout: Duration, params: Vec, priority: Priority, result_tx: ResultSender, ) -> Result<(), String> { self.to_host_tx - .send(ToHost::ExecutePvf { pvf, params, priority, result_tx }) + .send(ToHost::ExecutePvf { pvf, execution_timeout, params, priority, result_tx }) .await .map_err(|_| "the inner loop hung up".to_string()) } @@ -83,8 +84,16 @@ impl ValidationHost { } enum ToHost { - ExecutePvf { pvf: Pvf, params: Vec, priority: Priority, result_tx: ResultSender }, - HeadsUp { active_pvfs: Vec }, + ExecutePvf { + pvf: Pvf, + execution_timeout: Duration, + params: Vec, + priority: Priority, + result_tx: ResultSender, + }, + HeadsUp { + active_pvfs: Vec, + }, } /// Configuration for the validation host. @@ -200,6 +209,7 @@ pub fn start(config: Config, metrics: Metrics) -> (ValidationHost, impl Future, result_tx: ResultSender, } @@ -210,11 +220,18 @@ struct PendingExecutionRequest { struct AwaitingPrepare(HashMap>); impl AwaitingPrepare { - fn add(&mut self, artifact_id: ArtifactId, params: Vec, result_tx: ResultSender) { - self.0 - .entry(artifact_id) - .or_default() - .push(PendingExecutionRequest { params, result_tx }); + fn add( + &mut self, + artifact_id: ArtifactId, + execution_timeout: Duration, + params: Vec, + result_tx: ResultSender, + ) { + self.0.entry(artifact_id).or_default().push(PendingExecutionRequest { + execution_timeout, + params, + result_tx, + }); } fn take(&mut self, artifact_id: &ArtifactId) -> Vec { @@ -360,7 +377,7 @@ async fn handle_to_host( to_host: ToHost, ) -> Result<(), Fatal> { match to_host { - ToHost::ExecutePvf { pvf, params, priority, result_tx } => { + ToHost::ExecutePvf { pvf, execution_timeout, params, priority, result_tx } => { handle_execute_pvf( cache_path, artifacts, @@ -368,6 +385,7 @@ async fn handle_to_host( execute_queue, awaiting_prepare, pvf, + execution_timeout, params, priority, result_tx, @@ -389,6 +407,7 @@ async fn handle_execute_pvf( execute_queue: &mut mpsc::Sender, awaiting_prepare: &mut AwaitingPrepare, pvf: Pvf, + execution_timeout: Duration, params: Vec, priority: Priority, result_tx: ResultSender, @@ -404,6 +423,7 @@ async fn handle_execute_pvf( execute_queue, execute::ToQueue::Enqueue { artifact: ArtifactPathId::new(artifact_id, cache_path), + execution_timeout, params, result_tx, }, @@ -417,7 +437,7 @@ async fn handle_execute_pvf( ) .await?; - awaiting_prepare.add(artifact_id, params, result_tx); + awaiting_prepare.add(artifact_id, execution_timeout, params, result_tx); }, } } else { @@ -426,7 +446,7 @@ async fn handle_execute_pvf( artifacts.insert_preparing(artifact_id.clone()); send_prepare(prepare_queue, prepare::ToQueue::Enqueue { priority, pvf }).await?; - awaiting_prepare.add(artifact_id, params, result_tx); + awaiting_prepare.add(artifact_id, execution_timeout, params, result_tx); } return Ok(()) @@ -499,7 +519,7 @@ async fn handle_prepare_done( // It's finally time to dispatch all the execution requests that were waiting for this artifact // to be prepared. let pending_requests = awaiting_prepare.take(&artifact_id); - for PendingExecutionRequest { params, result_tx } in pending_requests { + for PendingExecutionRequest { execution_timeout, params, result_tx } in pending_requests { if result_tx.is_canceled() { // Preparation could've taken quite a bit of time and the requester may be not interested // in execution anymore, in which case we just skip the request. @@ -510,6 +530,7 @@ async fn handle_prepare_done( execute_queue, execute::ToQueue::Enqueue { artifact: ArtifactPathId::new(artifact_id.clone(), cache_path), + execution_timeout, params, result_tx, }, @@ -597,6 +618,8 @@ mod tests { use assert_matches::assert_matches; use futures::future::BoxFuture; + const TEST_EXECUTION_TIMEOUT: Duration = Duration::from_secs(3); + #[async_std::test] async fn pulse_test() { let pulse = pulse_every(Duration::from_millis(100)); @@ -840,9 +863,15 @@ mod tests { .await; let (result_tx, _result_rx) = oneshot::channel(); - host.execute_pvf(Pvf::from_discriminator(1), vec![], Priority::Critical, result_tx) - .await - .unwrap(); + host.execute_pvf( + Pvf::from_discriminator(1), + TEST_EXECUTION_TIMEOUT, + vec![], + Priority::Critical, + result_tx, + ) + .await + .unwrap(); run_until( &mut test.run, @@ -862,13 +891,20 @@ mod tests { let mut host = test.host_handle(); let (result_tx, result_rx_pvf_1_1) = oneshot::channel(); - host.execute_pvf(Pvf::from_discriminator(1), b"pvf1".to_vec(), Priority::Normal, result_tx) - .await - .unwrap(); + host.execute_pvf( + Pvf::from_discriminator(1), + TEST_EXECUTION_TIMEOUT, + b"pvf1".to_vec(), + Priority::Normal, + result_tx, + ) + .await + .unwrap(); let (result_tx, result_rx_pvf_1_2) = oneshot::channel(); host.execute_pvf( Pvf::from_discriminator(1), + TEST_EXECUTION_TIMEOUT, b"pvf1".to_vec(), Priority::Critical, result_tx, @@ -877,9 +913,15 @@ mod tests { .unwrap(); let (result_tx, result_rx_pvf_2) = oneshot::channel(); - host.execute_pvf(Pvf::from_discriminator(2), b"pvf2".to_vec(), Priority::Normal, result_tx) - .await - .unwrap(); + host.execute_pvf( + Pvf::from_discriminator(2), + TEST_EXECUTION_TIMEOUT, + b"pvf2".to_vec(), + Priority::Normal, + result_tx, + ) + .await + .unwrap(); assert_matches!( test.poll_and_recv_to_prepare_queue().await, @@ -947,9 +989,15 @@ mod tests { let mut host = test.host_handle(); let (result_tx, result_rx) = oneshot::channel(); - host.execute_pvf(Pvf::from_discriminator(1), b"pvf1".to_vec(), Priority::Normal, result_tx) - .await - .unwrap(); + host.execute_pvf( + Pvf::from_discriminator(1), + TEST_EXECUTION_TIMEOUT, + b"pvf1".to_vec(), + Priority::Normal, + result_tx, + ) + .await + .unwrap(); assert_matches!( test.poll_and_recv_to_prepare_queue().await, diff --git a/node/core/pvf/tests/it/main.rs b/node/core/pvf/tests/it/main.rs index 3689217880ef..e8fd7b665aa3 100644 --- a/node/core/pvf/tests/it/main.rs +++ b/node/core/pvf/tests/it/main.rs @@ -20,11 +20,13 @@ use polkadot_node_core_pvf::{ start, Config, InvalidCandidate, Metrics, Pvf, ValidationError, ValidationHost, }; use polkadot_parachain::primitives::{BlockData, ValidationParams, ValidationResult}; +use std::time::Duration; mod adder; mod worker_common; const PUPPET_EXE: &str = env!("CARGO_BIN_EXE_puppet_worker"); +const TEST_EXECUTION_TIMEOUT: Duration = Duration::from_secs(3); struct TestHost { _cache_dir: tempfile::TempDir, @@ -64,6 +66,7 @@ impl TestHost { .await .execute_pvf( Pvf::from_code(code.into()), + TEST_EXECUTION_TIMEOUT, params.encode(), polkadot_node_core_pvf::Priority::Normal, result_tx, diff --git a/node/overseer/examples/minimal-example.rs b/node/overseer/examples/minimal-example.rs index 0ff8201594fb..41ec66642976 100644 --- a/node/overseer/examples/minimal-example.rs +++ b/node/overseer/examples/minimal-example.rs @@ -75,6 +75,7 @@ impl Subsystem1 { let msg = CandidateValidationMessage::ValidateFromChainState( Default::default(), PoV { block_data: BlockData(Vec::new()) }.into(), + Default::default(), tx, ); ctx.send_message(::AllMessages::from(msg)) diff --git a/node/overseer/src/tests.rs b/node/overseer/src/tests.rs index 39eb91e0f6d6..7fe1ed701a83 100644 --- a/node/overseer/src/tests.rs +++ b/node/overseer/src/tests.rs @@ -112,6 +112,7 @@ where ctx.send_message(CandidateValidationMessage::ValidateFromChainState( Default::default(), PoV { block_data: BlockData(Vec::new()) }.into(), + Default::default(), tx, )) .await; @@ -791,7 +792,12 @@ where fn test_candidate_validation_msg() -> CandidateValidationMessage { let (sender, _) = oneshot::channel(); let pov = Arc::new(PoV { block_data: BlockData(Vec::new()) }); - CandidateValidationMessage::ValidateFromChainState(Default::default(), pov, sender) + CandidateValidationMessage::ValidateFromChainState( + Default::default(), + pov, + Default::default(), + sender, + ) } fn test_candidate_backing_msg() -> CandidateBackingMessage { diff --git a/node/primitives/src/lib.rs b/node/primitives/src/lib.rs index 6e8490b67b91..adc8846f4298 100644 --- a/node/primitives/src/lib.rs +++ b/node/primitives/src/lib.rs @@ -22,7 +22,7 @@ #![deny(missing_docs)] -use std::{convert::TryFrom, pin::Pin}; +use std::{convert::TryFrom, pin::Pin, time::Duration}; use bounded_vec::BoundedVec; use futures::Future; @@ -71,6 +71,17 @@ pub const POV_BOMB_LIMIT: usize = (MAX_POV_SIZE * 4u32) as usize; /// Number of sessions we want to consider in disputes. pub const DISPUTE_WINDOW: SessionIndex = 6; +/// The amount of time to spend on execution during backing. +pub const BACKING_EXECUTION_TIMEOUT: Duration = Duration::from_secs(2); + +/// The amount of time to spend on execution during approval or disputes. +/// +/// This is deliberately much longer than the backing execution timeout to +/// ensure that in the absence of extremely large disparities between hardware, +/// blocks that pass backing are considerd executable by approval checkers or +/// dispute participants. +pub const APPROVAL_EXECUTION_TIMEOUT: Duration = Duration::from_secs(6); + /// The cumulative weight of a block in a fork-choice rule. pub type BlockWeight = u32; diff --git a/node/subsystem-types/src/messages.rs b/node/subsystem-types/src/messages.rs index c6fb92736c51..ea70d3b2707a 100644 --- a/node/subsystem-types/src/messages.rs +++ b/node/subsystem-types/src/messages.rs @@ -51,6 +51,7 @@ use polkadot_statement_table::v1::Misbehavior; use std::{ collections::{BTreeMap, HashSet}, sync::Arc, + time::Duration, }; /// Network events as transmitted to other subsystems, wrapped in their message types. @@ -114,6 +115,8 @@ pub enum CandidateValidationMessage { ValidateFromChainState( CandidateDescriptor, Arc, + /// Execution timeout + Duration, oneshot::Sender>, ), /// Validate a candidate with provided, exhaustive parameters for validation. @@ -130,6 +133,8 @@ pub enum CandidateValidationMessage { ValidationCode, CandidateDescriptor, Arc, + /// Execution timeout + Duration, oneshot::Sender>, ), } @@ -138,8 +143,8 @@ impl CandidateValidationMessage { /// If the current variant contains the relay parent hash, return it. pub fn relay_parent(&self) -> Option { match self { - Self::ValidateFromChainState(_, _, _) => None, - Self::ValidateFromExhaustive(_, _, _, _, _) => None, + Self::ValidateFromChainState(_, _, _, _) => None, + Self::ValidateFromExhaustive(_, _, _, _, _, _) => None, } } } diff --git a/roadmap/implementers-guide/src/node/approval/approval-voting.md b/roadmap/implementers-guide/src/node/approval/approval-voting.md index c6367c050a04..adb95e1f6389 100644 --- a/roadmap/implementers-guide/src/node/approval/approval-voting.md +++ b/roadmap/implementers-guide/src/node/approval/approval-voting.md @@ -281,7 +281,7 @@ On receiving an `ApprovedAncestor(Hash, BlockNumber, response_channel)`: * Load the historical validation code of the parachain by dispatching a `RuntimeApiRequest::ValidationCodeByHash(descriptor.validation_code_hash)` against the state of `block_hash`. * Spawn a background task with a clone of `background_tx` * Wait for the available data - * Issue a `CandidateValidationMessage::ValidateFromExhaustive` message + * Issue a `CandidateValidationMessage::ValidateFromExhaustive` message with `APPROVAL_EXECUTION_TIMEOUT` as the timeout parameter. * Wait for the result of validation * Check that the result of validation, if valid, matches the commitments in the receipt. * If valid, issue a message on `background_tx` detailing the request. diff --git a/roadmap/implementers-guide/src/node/backing/candidate-backing.md b/roadmap/implementers-guide/src/node/backing/candidate-backing.md index c39ffabdcd98..4b25a89cb1a5 100644 --- a/roadmap/implementers-guide/src/node/backing/candidate-backing.md +++ b/roadmap/implementers-guide/src/node/backing/candidate-backing.md @@ -123,7 +123,7 @@ Dispatch a [`AvailabilityDistributionMessage`][ADM]`::FetchPoV{ validator_index, ### Validate PoV Block Create a `(sender, receiver)` pair. -Dispatch a `CandidateValidationMessage::Validate(validation function, candidate, pov, sender)` and listen on the receiver for a response. +Dispatch a `CandidateValidationMessage::Validate(validation function, candidate, pov, BACKING_EXECUTION_TIMEOUT, sender)` and listen on the receiver for a response. ### Distribute Signed Statement diff --git a/roadmap/implementers-guide/src/node/disputes/dispute-participation.md b/roadmap/implementers-guide/src/node/disputes/dispute-participation.md index b3e1c11fa2be..fc0517fa4e1e 100644 --- a/roadmap/implementers-guide/src/node/disputes/dispute-participation.md +++ b/roadmap/implementers-guide/src/node/disputes/dispute-participation.md @@ -48,7 +48,7 @@ Conclude. * If the data is recovered, dispatch a [`RuntimeApiMessage::ValidationCodeByHash`][RuntimeApiMessage] with the parameters `(candidate_receipt.descriptor.validation_code_hash)` at `state.recent_block.hash`. * Dispatch a [`AvailabilityStoreMessage::StoreAvailableData`][AvailabilityStoreMessage] with the data. * If the code is not fetched from the chain, return. This should be impossible with correct relay chain configuration, at least if chain synchronization is working correctly. -* Dispatch a [`CandidateValidationMessage::ValidateFromExhaustive`][CandidateValidationMessage] with the available data and the validation code. +* Dispatch a [`CandidateValidationMessage::ValidateFromExhaustive`][CandidateValidationMessage] with the available data and the validation code and `APPROVAL_EXECUTION_TIMEOUT` as the timeout parameter. * If the validation result is `Invalid`, [cast invalid votes](#cast-votes) and return. * If the validation fails, [cast invalid votes](#cast-votes) and return. * If the validation succeeds, compute the `CandidateCommitments` based on the validation result and compare against the candidate receipt's `commitments_hash`. If they match, [cast valid votes](#cast-votes) and if not, [cast invalid votes](#cast-votes). diff --git a/roadmap/implementers-guide/src/types/overseer-protocol.md b/roadmap/implementers-guide/src/types/overseer-protocol.md index 78d536f1a21c..f3195e713399 100644 --- a/roadmap/implementers-guide/src/types/overseer-protocol.md +++ b/roadmap/implementers-guide/src/types/overseer-protocol.md @@ -785,6 +785,9 @@ enum ValidationResult { Invalid, } +const BACKING_EXECUTION_TIMEOUT: Duration = 2 seconds; +const APPROVAL_EXECUTION_TIMEOUT: Duration = 6 seconds; + /// Messages received by the Validation subsystem. /// /// ## Validation Requests @@ -807,6 +810,7 @@ pub enum CandidateValidationMessage { ValidateFromChainState( CandidateDescriptor, Arc, + Duration, // Execution timeout. oneshot::Sender>, ), /// Validate a candidate with provided, exhaustive parameters for validation. @@ -823,6 +827,7 @@ pub enum CandidateValidationMessage { ValidationCode, CandidateDescriptor, Arc, + Duration, // Execution timeout. oneshot::Sender>, ), } From 1a9239152b6c7b4a9f2f05e6dea51f1289715223 Mon Sep 17 00:00:00 2001 From: Robert Habermeier Date: Mon, 4 Oct 2021 17:02:31 +0200 Subject: [PATCH 2/3] Free disputed cores before processing bitfields (#4008) * guide: extract free_cores in scheduler * scheduler: extract free cores to a separate function * guide: remove disputed cores from scheduler first * free disputed cores in scheduler before processing bitfields * spellcheck is mostly right but sometimes stupid * add comment and fmt --- .../src/runtime/parainherent.md | 4 +- .../src/runtime/scheduler.md | 8 ++- runtime/parachains/src/paras_inherent.rs | 16 +++-- runtime/parachains/src/scheduler.rs | 71 ++++++++++--------- 4 files changed, 58 insertions(+), 41 deletions(-) diff --git a/roadmap/implementers-guide/src/runtime/parainherent.md b/roadmap/implementers-guide/src/runtime/parainherent.md index cb5bb45d8d81..f9aacc2c3578 100644 --- a/roadmap/implementers-guide/src/runtime/parainherent.md +++ b/roadmap/implementers-guide/src/runtime/parainherent.md @@ -26,11 +26,11 @@ Included: Option<()>, 1. Hash the parent header and make sure that it corresponds to the block hash of the parent (tracked by the `frame_system` FRAME module), 1. Invoke `Disputes::provide_multi_dispute_data`. 1. If `Disputes::is_frozen`, return and set `Included` to `Some(())`. - 1. If there are any concluded disputes from the current session, invoke `Inclusion::collect_disputed` with the disputed candidates. Annotate each returned core with `FreedReason::Concluded`. + 1. If there are any concluded disputes from the current session, invoke `Inclusion::collect_disputed` with the disputed candidates. Annotate each returned core with `FreedReason::Concluded`, sort them, and invoke `Scheduler::free_cores` with them. 1. The `Bitfields` are first forwarded to the `Inclusion::process_bitfields` routine, returning a set of freed cores. Provide the number of availability cores (`Scheduler::availability_cores().len()`) as the expected number of bits and a `Scheduler::core_para` as a core-lookup to the `process_bitfields` routine. Annotate each of these freed cores with `FreedReason::Concluded`. 1. For each freed candidate from the `Inclusion::process_bitfields` call, invoke `Disputes::note_included(current_session, candidate)`. 1. If `Scheduler::availability_timeout_predicate` is `Some`, invoke `Inclusion::collect_pending` using it and annotate each of those freed cores with `FreedReason::TimedOut`. - 1. Combine and sort the dispute-freed cores, the bitfield-freed cores, and the timed-out cores. + 1. Combine and sort the the bitfield-freed cores and the timed-out cores. 1. Invoke `Scheduler::clear` 1. Invoke `Scheduler::schedule(freed_cores, System::current_block())` 1. Extract `parent_storage_root` from the parent header, diff --git a/roadmap/implementers-guide/src/runtime/scheduler.md b/roadmap/implementers-guide/src/runtime/scheduler.md index 68b1a8abb722..16c3280d1808 100644 --- a/roadmap/implementers-guide/src/runtime/scheduler.md +++ b/roadmap/implementers-guide/src/runtime/scheduler.md @@ -82,7 +82,7 @@ digraph { ## Validator Groups -Validator group assignments do not need to change very quickly. The security benefits of fast rotation are redundant with the challenge mechanism in the [Approval process](../protocol-approval.md). Because of this, we only divide validators into groups at the beginning of the session and do not shuffle membership during the session. However, we do take steps to ensure that no particular validator group has dominance over a single parachain or parathread-multiplexer for an entire session to provide better guarantees of liveness. +Validator group assignments do not need to change very quickly. The security benefits of fast rotation are redundant with the challenge mechanism in the [Approval process](../protocol-approval.md). Because of this, we only divide validators into groups at the beginning of the session and do not shuffle membership during the session. However, we do take steps to ensure that no particular validator group has dominance over a single parachain or parathread-multiplexer for an entire session to provide better guarantees of live-ness. Validator groups rotate across availability cores in a round-robin fashion, with rotation occurring at fixed intervals. The i'th group will be assigned to the `(i+k)%n`'th core at any point in time, where `k` is the number of rotations that have occurred in the session, and `n` is the number of cores. This makes upcoming rotations within the same session predictable. @@ -185,7 +185,7 @@ Actions: 1. Resize `AvailabilityCores` to have length `n_cores` with all `None` entries. 1. Compute new validator groups by shuffling using a secure randomness beacon - Note that the total number of validators `V` in AV may not be evenly divided by `n_cores`. - - The groups are selected by partitioning AV. The first V % N groups will have (V / n_cores) + 1 members, while the remaining groups will have (V / N) members each. + - The groups are selected by partitioning AV. The first `V % N` groups will have `(V / n_cores) + 1` members, while the remaining groups will have `(V / N)` members each. - Instead of using the indices within AV, which point to the broader set, indices _into_ AV should be used. This implies that groups should have simply ascending validator indices. 1. Prune the parathread queue to remove all retries beyond `configuration.parathread_retries`. - Also prune all parathread claims corresponding to de-registered parathreads. @@ -209,11 +209,13 @@ No finalization routine runs for this module. - The core used for the parathread claim is the `next_core` field of the `ParathreadQueue` and adding `Paras::parachains().len()` to it. - `next_core` is then updated by adding 1 and taking it modulo `config.parathread_cores`. - The claim is then added to the claim index. -- `schedule(Vec<(CoreIndex, FreedReason)>, now: BlockNumber)`: schedule new core assignments, with a parameter indicating previously-occupied cores which are to be considered returned and why they are being returned. +- `free_cores(Vec<(CoreIndex, FreedReason)>)`: indicate previosuly-occupied cores which are to be considered returned and why they are being returned. - All freed parachain cores should be assigned to their respective parachain - All freed parathread cores whose reason for freeing was `FreedReason::Concluded` should have the claim removed from the claim index. - All freed parathread cores whose reason for freeing was `FreedReason::TimedOut` should have the claim added to the parathread queue again without retries incremented - All freed parathread cores should take the next parathread entry from the queue. +- `schedule(Vec<(CoreIndex, FreedReason)>, now: BlockNumber)`: schedule new core assignments, with a parameter indicating previously-occupied cores which are to be considered returned and why they are being returned. + - Invoke `free_cores(freed_cores)` - The i'th validator group will be assigned to the `(i+k)%n`'th core at any point in time, where `k` is the number of rotations that have occurred in the session, and `n` is the total number of cores. This makes upcoming rotations within the same session predictable. Rotations are based off of `now`. - `scheduled() -> Vec`: Get currently scheduled core assignments. - `occupied(Vec)`. Note that the given cores have become occupied. diff --git a/runtime/parachains/src/paras_inherent.rs b/runtime/parachains/src/paras_inherent.rs index ea480ad7c96a..cbffb9ff7937 100644 --- a/runtime/parachains/src/paras_inherent.rs +++ b/runtime/parachains/src/paras_inherent.rs @@ -173,7 +173,7 @@ pub mod pallet { // Handle disputes logic. let current_session = >::session_index(); - let freed_disputed: Vec<(_, FreedReason)> = { + { let new_current_dispute_sets: Vec<_> = disputes .iter() .filter(|s| s.session == current_session) @@ -187,7 +187,7 @@ pub mod pallet { return Ok(Some(MINIMAL_INCLUSION_INHERENT_WEIGHT).into()) } - if !new_current_dispute_sets.is_empty() { + let mut freed_disputed = if !new_current_dispute_sets.is_empty() { let concluded_invalid_disputes: Vec<_> = new_current_dispute_sets .iter() .filter(|(s, c)| T::DisputesHandler::concluded_invalid(*s, *c)) @@ -200,6 +200,13 @@ pub mod pallet { .collect() } else { Vec::new() + }; + + if !freed_disputed.is_empty() { + // unstable sort is fine, because core indices are unique + // i.e. the same candidate can't occupy 2 cores at once. + freed_disputed.sort_unstable_by_key(|pair| pair.0); // sort by core index + >::free_cores(freed_disputed); } }; @@ -227,12 +234,13 @@ pub mod pallet { }; // Schedule paras again, given freed cores, and reasons for freeing. - let mut freed = freed_disputed + let mut freed = freed_concluded .into_iter() - .chain(freed_concluded.into_iter().map(|(c, _hash)| (c, FreedReason::Concluded))) + .map(|(c, _hash)| (c, FreedReason::Concluded)) .chain(freed_timeout.into_iter().map(|c| (c, FreedReason::TimedOut))) .collect::>(); + // unstable sort is fine, because core indices are unique. freed.sort_unstable_by_key(|pair| pair.0); // sort by core index >::clear(); diff --git a/runtime/parachains/src/scheduler.rs b/runtime/parachains/src/scheduler.rs index e6772c19d910..8e948e3b5529 100644 --- a/runtime/parachains/src/scheduler.rs +++ b/runtime/parachains/src/scheduler.rs @@ -369,6 +369,43 @@ impl Pallet { }) } + /// Free unassigned cores. Provide a list of cores that should be considered newly-freed along with the reason + /// for them being freed. The list is assumed to be sorted in ascending order by core index. + pub(crate) fn free_cores(just_freed_cores: impl IntoIterator) { + let config = >::config(); + + AvailabilityCores::::mutate(|cores| { + for (freed_index, freed_reason) in just_freed_cores { + if (freed_index.0 as usize) < cores.len() { + match cores[freed_index.0 as usize].take() { + None => continue, + Some(CoreOccupied::Parachain) => {}, + Some(CoreOccupied::Parathread(entry)) => { + match freed_reason { + FreedReason::Concluded => { + // After a parathread candidate has successfully been included, + // open it up for further claims! + ParathreadClaimIndex::::mutate(|index| { + if let Ok(i) = index.binary_search(&entry.claim.0) { + index.remove(i); + } + }) + }, + FreedReason::TimedOut => { + // If a parathread candidate times out, it's not the collator's fault, + // so we don't increment retries. + ParathreadQueue::::mutate(|queue| { + queue.enqueue_entry(entry, config.parathread_cores); + }) + }, + } + }, + } + } + } + }) + } + /// Schedule all unassigned cores, where possible. Provide a list of cores that should be considered /// newly-freed along with the reason for them being freed. The list is assumed to be sorted in /// ascending order by core index. @@ -376,38 +413,9 @@ impl Pallet { just_freed_cores: impl IntoIterator, now: T::BlockNumber, ) { - let mut cores = AvailabilityCores::::get(); - let config = >::config(); - - for (freed_index, freed_reason) in just_freed_cores { - if (freed_index.0 as usize) < cores.len() { - match cores[freed_index.0 as usize].take() { - None => continue, - Some(CoreOccupied::Parachain) => {}, - Some(CoreOccupied::Parathread(entry)) => { - match freed_reason { - FreedReason::Concluded => { - // After a parathread candidate has successfully been included, - // open it up for further claims! - ParathreadClaimIndex::::mutate(|index| { - if let Ok(i) = index.binary_search(&entry.claim.0) { - index.remove(i); - } - }) - }, - FreedReason::TimedOut => { - // If a parathread candidate times out, it's not the collator's fault, - // so we don't increment retries. - ParathreadQueue::::mutate(|queue| { - queue.enqueue_entry(entry, config.parathread_cores); - }) - }, - } - }, - } - } - } + Self::free_cores(just_freed_cores); + let cores = AvailabilityCores::::get(); let parachains = >::parachains(); let mut scheduled = Scheduled::::get(); let mut parathread_queue = ParathreadQueue::::get(); @@ -510,7 +518,6 @@ impl Pallet { Scheduled::::set(scheduled); ParathreadQueue::::set(parathread_queue); - AvailabilityCores::::set(cores); } /// Note that the given cores have become occupied. Behavior undefined if any of the given cores were not scheduled From 4870e7bc52e305fc79e5b612524e1e5e7667212c Mon Sep 17 00:00:00 2001 From: Chevdor Date: Mon, 4 Oct 2021 18:09:19 +0200 Subject: [PATCH 3/3] chore: update `builder` image (#3884) * chore: update `builder` image fix #9765 * fix: fix binary path * chore: refresh of the builder image + build script * Fix port * Fix image source * Update scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile Co-authored-by: Denis Pisarev --- scripts/dockerfiles/polkadot/build.sh | 8 ++--- .../polkadot/docker-compose-local.yml | 30 +++++++++++-------- .../dockerfiles/polkadot/docker-compose.yml | 17 +++++++---- .../polkadot/polkadot_builder.Dockerfile | 26 ++++++++++------ 4 files changed, 49 insertions(+), 32 deletions(-) diff --git a/scripts/dockerfiles/polkadot/build.sh b/scripts/dockerfiles/polkadot/build.sh index ee4d29991a86..cf100bedbfaa 100755 --- a/scripts/dockerfiles/polkadot/build.sh +++ b/scripts/dockerfiles/polkadot/build.sh @@ -8,19 +8,17 @@ PROJECT_ROOT=`git rev-parse --show-toplevel` cd $PROJECT_ROOT # Find the current version from Cargo.toml -VERSION=`grep "^version" ./Cargo.toml | egrep -o "([0-9\.]+)"` +VERSION=`grep "^version" ./cli/Cargo.toml | egrep -o "([0-9\.]+)"` GITUSER=parity GITREPO=polkadot # Build the image echo "Building ${GITUSER}/${GITREPO}:latest docker image, hang on!" -time docker build -f ./scripts/docker/polkadot/polkadot_builder.Dockerfile --build-arg RUSTC_WRAPPER= --build-arg PROFILE=release -t ${GITUSER}/${GITREPO}:latest . +time docker build -f ./scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile -t ${GITUSER}/${GITREPO}:latest . +docker tag ${GITUSER}/${GITREPO}:latest ${GITUSER}/${GITREPO}:v${VERSION} # Show the list of available images for this repo echo "Image is ready" docker images | grep ${GITREPO} -echo -e "\nIf you just built version ${VERSION}, you may want to update your tag:" -echo " $ docker tag ${GITUSER}/${GITREPO}:$VERSION ${GITUSER}/${GITREPO}:${VERSION}" - popd diff --git a/scripts/dockerfiles/polkadot/docker-compose-local.yml b/scripts/dockerfiles/polkadot/docker-compose-local.yml index 2c93799259ef..1ff3a1ccaac2 100644 --- a/scripts/dockerfiles/polkadot/docker-compose-local.yml +++ b/scripts/dockerfiles/polkadot/docker-compose-local.yml @@ -1,35 +1,39 @@ version: '3' services: node_alice: - build: - context: . - dockerfile: polkadot_builder.Dockerfile ports: - "30333:30333" - "9933:9933" - "9944:9944" - image: chevdor/polkadot:latest + - "9615:9615" + image: parity/polkadot:latest volumes: - "polkadot-data-alice:/data" - command: polkadot --chain=polkadot-local --alice -d /data --node-key 0000000000000000000000000000000000000000000000000000000000000001 + command: | + --chain=polkadot-local + --alice + -d /data + --node-key 0000000000000000000000000000000000000000000000000000000000000001 networks: testing_net: ipv4_address: 172.28.1.1 node_bob: - build: - context: . - dockerfile: polkadot_builder.Dockerfile ports: - - "30344:30344" - - "9935:9935" - - "9945:9945" - image: chevdor/polkadot:latest + - "30344:30333" + - "9935:9933" + - "9945:9944" + - "29615:9615" + image: parity/polkadot:latest volumes: - "polkadot-data-bob:/data" links: - "node_alice:alice" - command: polkadot --chain=polkadot-local --bob -d /data --port 30344 --rpc-port 9935 --ws-port 9945 --bootnodes '/ip4/172.28.1.1/tcp/30333/p2p/QmRpheLN4JWdAnY7HGJfWFNbfkQCb6tFf4vvA6hgjMZKrR' + command: | + --chain=polkadot-local + --bob + -d /data + --bootnodes '/ip4/172.28.1.1/tcp/30333/p2p/QmRpheLN4JWdAnY7HGJfWFNbfkQCb6tFf4vvA6hgjMZKrR' networks: testing_net: ipv4_address: 172.28.1.2 diff --git a/scripts/dockerfiles/polkadot/docker-compose.yml b/scripts/dockerfiles/polkadot/docker-compose.yml index 54fe158735d8..978191af88c1 100644 --- a/scripts/dockerfiles/polkadot/docker-compose.yml +++ b/scripts/dockerfiles/polkadot/docker-compose.yml @@ -1,16 +1,23 @@ version: '3' services: polkadot: - build: - context: . - dockerfile: polkadot_builder.Dockerfile ports: - "127.0.0.1:30333:30333/tcp" - "127.0.0.1:9933:9933/tcp" - image: chevdor/polkadot:latest + image: parity/polkadot:latest volumes: - "polkadot-data:/data" - command: polkadot + command: | + --unsafe-rpc-external + --unsafe-ws-external + --rpc-cors all + --prometheus-external + + ports: + - "30333:30333" + - "9933:9933" + - "9944:9944" + - "9615:9615" volumes: polkadot-data: diff --git a/scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile b/scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile index 2fc5787e6cb0..6b096244a506 100644 --- a/scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile +++ b/scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile @@ -1,27 +1,35 @@ +# This is the build stage for Polkadot. Here we create the binary in a temporary image. FROM docker.io/paritytech/ci-linux:production as builder -LABEL io.parity.image.description="This is the build stage for Polkadot. Here we create the binary." WORKDIR /polkadot - COPY . /polkadot -RUN cargo build --release --locked - -# ===== SECOND STAGE ====== +RUN cargo build --locked --release +# This is the 2nd stage: a very small image where we copy the Polkadot binary." FROM docker.io/library/ubuntu:20.04 -LABEL io.parity.image.description="Polkadot: a platform for web3. This is a self-buit multistage image." + +LABEL description="Multistage Docker image for Polkadot: a platform for web3" \ + io.parity.image.type="builder" \ + io.parity.image.authors="chevdor@gmail.com, devops-team@parity.io" \ + io.parity.image.vendor="Parity Technologies" \ + io.parity.image.description="Polkadot: a platform for web3" \ + io.parity.image.source="https://github.com/paritytech/polkadot/blob/${VCS_REF}/scripts/dockerfiles/polkadot/polkadot_builder.Dockerfile" \ + io.parity.image.documentation="https://github.com/paritytech/polkadot/" COPY --from=builder /polkadot/target/release/polkadot /usr/local/bin RUN useradd -m -u 1000 -U -s /bin/sh -d /polkadot polkadot && \ - mkdir -p /polkadot/.local/share && \ - mkdir /data && \ + mkdir -p /data /polkadot/.local/share && \ chown -R polkadot:polkadot /data && \ ln -s /data /polkadot/.local/share/polkadot && \ - rm -rf /usr/bin /usr/sbin +# unclutter and minimize the attack surface + rm -rf /usr/bin /usr/sbin && \ +# check if executable works in this container + /usr/local/bin/polkadot --version USER polkadot + EXPOSE 30333 9933 9944 9615 VOLUME ["/data"]