Commit 05583c34 authored by asynchronous rob's avatar asynchronous rob Committed by Gav Wood
Browse files

Proposal creation and evaluation to plug into BFT (#77)

* reshuffle consensus libraries

* polkadot-useful type definitions for statement table

* begin BftService

* primary selection logic

* bft service implementation without I/O

* extract out `BlockImport` trait

* allow bft primitives to compile on wasm

* Block builder (substrate)

* take polkadot-consensus down to the core.

* test for preemption

* fix test build

* Fix wasm build

* Bulid on any block

* Test for block builder.

* Block import tests for client.

* Tidy ups

* clean up block builder instantiation

* justification verification logic

* JustifiedHeader and import

* Propert block generation for tests

* network and tablerouter trait

* use statement import to drive creation of further statements

* Fixed rpc tests

* custom error type for consensus

* create proposer

* asynchronous proposal evaluation

* inherent transactions in polkadot runtime

* fix tests to match real polkadot block constraints

* implicitly generate inherent functions

* add inherent transaction functionality to block body

* block builder logic for polkadot

* some tests for the polkadot API
parent 1308aab5
......@@ -11,3 +11,6 @@ polkadot-primitives = { path = "../primitives" }
substrate-client = { path = "../../substrate/client" }
substrate-executor = { path = "../../substrate/executor" }
substrate-state-machine = { path = "../../substrate/state-machine" }
[dev-dependencies]
substrate-keyring = { path = "../../substrate/keyring" }
......@@ -18,7 +18,7 @@
//! runtime.
extern crate polkadot_executor as polkadot_executor;
extern crate polkadot_runtime ;
extern crate polkadot_runtime;
extern crate polkadot_primitives as primitives;
extern crate substrate_client as client;
extern crate substrate_executor as substrate_executor;
......@@ -27,13 +27,18 @@ extern crate substrate_state_machine as state_machine;
#[macro_use]
extern crate error_chain;
#[cfg(test)]
extern crate substrate_keyring as keyring;
use client::backend::Backend;
use client::Client;
use polkadot_runtime::runtime;
use polkadot_executor::Executor as LocalDispatch;
use substrate_executor::{NativeExecutionDispatch, NativeExecutor};
use primitives::{AccountId, SessionKey};
use primitives::block::Id as BlockId;
use state_machine::OverlayedChanges;
use primitives::{AccountId, SessionKey, Timestamp};
use primitives::block::{Id as BlockId, Block, Header, Body};
use primitives::transaction::UncheckedTransaction;
use primitives::parachain::DutyRoster;
error_chain! {
......@@ -43,10 +48,21 @@ error_chain! {
description("Unknown runtime code")
display("Unknown runtime code")
}
/// Unknown block ID.
UnknownBlock(b: BlockId) {
description("Unknown block")
display("Unknown block")
}
/// Attempted to push an inherent transaction manually.
PushedInherentTransaction(tx: UncheckedTransaction) {
description("Attempted to push an inherent transaction to a block."),
display("Pushed inherent transaction to a block: {:?}", tx),
}
/// Badly-formed transaction.
BadlyFormedTransaction(tx: UncheckedTransaction) {
description("Attempted to push a badly-formed transaction to a block."),
display("Pushed badly-formed transaction to a block: {:?}", tx),
}
/// Some other error.
// TODO: allow to be specified as associated type of PolkadotApi
Other(e: Box<::std::error::Error + Send>) {
......@@ -60,10 +76,29 @@ error_chain! {
}
}
impl From<client::error::Error> for Error {
fn from(e: client::error::Error) -> Error {
match e {
client::error::Error(client::error::ErrorKind::UnknownBlock(b), _) => Error::from_kind(ErrorKind::UnknownBlock(b)),
other => Error::from_kind(ErrorKind::Other(Box::new(other) as Box<_>)),
}
}
}
pub trait BlockBuilder: Sized {
/// Push a non-inherent transaction.
fn push_transaction(&mut self, transaction: UncheckedTransaction) -> Result<()>;
/// Finalise the block.
fn bake(self) -> Block;
}
/// Trait encapsulating the Polkadot API.
///
/// All calls should fail when the exact runtime is unknown.
pub trait PolkadotApi {
type BlockBuilder: BlockBuilder;
/// Get session keys at a given block.
fn session_keys(&self, at: &BlockId) -> Result<Vec<SessionKey>>;
......@@ -72,24 +107,26 @@ pub trait PolkadotApi {
/// Get the authority duty roster at a block.
fn duty_roster(&self, at: &BlockId) -> Result<DutyRoster>;
}
fn convert_client_error(e: client::error::Error) -> Error {
match e {
client::error::Error(client::error::ErrorKind::UnknownBlock(b), _) => Error::from_kind(ErrorKind::UnknownBlock(b)),
other => Error::from_kind(ErrorKind::Other(Box::new(other) as Box<_>)),
}
/// Get the timestamp registered at a block.
fn timestamp(&self, at: &BlockId) -> Result<Timestamp>;
/// Evaluate a block and see if it gives an error.
fn evaluate_block(&self, at: &BlockId, block: Block) -> Result<()>;
/// Create a block builder on top of the parent block.
fn build_block(&self, parent: &BlockId, timestamp: u64) -> Result<Self::BlockBuilder>;
}
// set up the necessary scaffolding to execute the runtime.
macro_rules! with_runtime {
($client: ident, $at: expr, $exec: expr) => {{
// bail if the code is not the same as the natively linked.
if $client.code_at($at).map_err(convert_client_error)? != LocalDispatch::native_equivalent() {
if $client.code_at($at)? != LocalDispatch::native_equivalent() {
bail!(ErrorKind::UnknownRuntime);
}
$client.state_at($at).map_err(convert_client_error).and_then(|state| {
$client.state_at($at).map_err(Error::from).and_then(|state| {
let mut changes = Default::default();
let mut ext = state_machine::Ext {
overlay: &mut changes,
......@@ -104,6 +141,8 @@ macro_rules! with_runtime {
impl<B: Backend> PolkadotApi for Client<B, NativeExecutor<LocalDispatch>>
where ::client::error::Error: From<<<B as Backend>::State as state_machine::backend::Backend>::Error>
{
type BlockBuilder = ClientBlockBuilder<B::State>;
fn session_keys(&self, at: &BlockId) -> Result<Vec<SessionKey>> {
with_runtime!(self, at, ::runtime::consensus::authorities)
}
......@@ -115,4 +154,185 @@ impl<B: Backend> PolkadotApi for Client<B, NativeExecutor<LocalDispatch>>
fn duty_roster(&self, at: &BlockId) -> Result<DutyRoster> {
with_runtime!(self, at, ::runtime::parachains::calculate_duty_roster)
}
fn timestamp(&self, at: &BlockId) -> Result<Timestamp> {
with_runtime!(self, at, ::runtime::timestamp::get)
}
fn evaluate_block(&self, at: &BlockId, block: Block) -> Result<()> {
with_runtime!(self, at, || ::runtime::system::internal::execute_block(block))
}
fn build_block(&self, parent: &BlockId, timestamp: Timestamp) -> Result<Self::BlockBuilder> {
if self.code_at(parent)? != LocalDispatch::native_equivalent() {
bail!(ErrorKind::UnknownRuntime);
}
let header = Header {
parent_hash: self.block_hash_from_id(parent)?.ok_or(ErrorKind::UnknownBlock(*parent))?,
number: self.block_number_from_id(parent)?.ok_or(ErrorKind::UnknownBlock(*parent))? + 1,
state_root: Default::default(),
transaction_root: Default::default(),
digest: Default::default(),
};
let body = Body {
timestamp: timestamp,
transactions: Vec::new(),
};
let mut builder = ClientBlockBuilder {
parent: *parent,
changes: OverlayedChanges::default(),
state: self.state_at(parent)?,
header,
timestamp,
transactions: Vec::new(),
};
for inherent in body.inherent_transactions() {
builder.execute_transaction(inherent)?;
}
Ok(builder)
}
}
/// A polkadot block builder.
#[derive(Debug, Clone)]
pub struct ClientBlockBuilder<S> {
parent: BlockId,
changes: OverlayedChanges,
state: S,
header: Header,
timestamp: Timestamp,
transactions: Vec<UncheckedTransaction>,
}
impl<S: state_machine::Backend> ClientBlockBuilder<S>
where S::Error: Into<client::error::Error>
{
// executes a transaction, inherent or otherwise, without appending to the list
fn execute_transaction(&mut self, transaction: UncheckedTransaction) -> Result<()> {
if !transaction.is_well_formed() {
bail!(ErrorKind::BadlyFormedTransaction(transaction));
}
let mut ext = state_machine::Ext {
overlay: &mut self.changes,
backend: &self.state,
};
// TODO: avoid clone
let header = self.header.clone();
let result = ::substrate_executor::with_native_environment(
&mut ext,
move || runtime::system::internal::execute_transaction(transaction, header),
).map_err(Into::into);
match result {
Ok(header) => {
ext.overlay.commit_prospective();
self.header = header;
Ok(())
}
Err(e) => {
ext.overlay.discard_prospective();
Err(e)
}
}
}
}
impl<S: state_machine::Backend> BlockBuilder for ClientBlockBuilder<S>
where S::Error: Into<client::error::Error>
{
fn push_transaction(&mut self, transaction: UncheckedTransaction) -> Result<()> {
if transaction.transaction.function.is_inherent() {
bail!(ErrorKind::PushedInherentTransaction(transaction));
} else {
self.execute_transaction(transaction.clone())?;
self.transactions.push(transaction);
Ok(())
}
}
fn bake(mut self) -> Block {
let mut ext = state_machine::Ext {
overlay: &mut self.changes,
backend: &self.state,
};
let old_header = self.header;
let final_header = ::substrate_executor::with_native_environment(
&mut ext,
move || runtime::system::internal::finalise_block(old_header)
).expect("all inherent transactions pushed; all other transactions executed correctly; qed");
Block {
header: final_header,
body: Body {
timestamp: self.timestamp,
transactions: self.transactions,
}
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use client::in_mem::Backend as InMemory;
use polkadot_runtime::genesismap::{additional_storage_with_genesis, GenesisConfig};
use substrate_executor::NativeExecutionDispatch;
use keyring::Keyring;
fn validators() -> Vec<AccountId> {
vec![
Keyring::One.to_raw_public(),
Keyring::Two.to_raw_public(),
]
}
fn client() -> Client<InMemory, NativeExecutor<LocalDispatch>> {
::client::new_in_mem(
LocalDispatch::new(),
|| {
let config = GenesisConfig::new_simple(validators(), 100);
// override code entry.
let mut storage = config.genesis_map();
storage.insert(b":code".to_vec(), LocalDispatch::native_equivalent().to_vec());
let block = ::client::genesis::construct_genesis_block(
&config.genesis_map()
);
storage.extend(additional_storage_with_genesis(&block));
(block.header, storage.into_iter().collect())
}
).unwrap()
}
#[test]
fn gets_session_and_validator_keys() {
let client = client();
assert_eq!(client.session_keys(&BlockId::Number(0)).unwrap(), validators());
assert_eq!(client.validators(&BlockId::Number(0)).unwrap(), validators());
}
#[test]
fn build_block() {
let client = client();
let block_builder = client.build_block(&BlockId::Number(0), 1_000_000).unwrap();
let block = block_builder.bake();
assert_eq!(block.header.number, 1);
}
#[test]
fn cannot_build_block_on_unknown_parent() {
let client = client();
assert!(client.build_block(&BlockId::Number(100), 1_000_000).is_err());
}
}
......@@ -61,7 +61,7 @@ pub trait ParachainContext {
fn produce_candidate<I: IntoIterator<Item=(ParaId, Message)>>(
&self,
ingress: I,
) -> (parachain::BlockData, polkadot_primitives::Signature);
) -> (parachain::BlockData, polkadot_primitives::AccountId, polkadot_primitives::Signature);
}
/// Relay chain context needed to collate.
......@@ -131,7 +131,7 @@ pub fn collate<'a, R, P>(local_id: ParaId, relay_context: R, para_context: P)
P: ParachainContext + 'a,
{
Box::new(collate_ingress(relay_context).map(move |ingress| {
let (block_data, signature) = para_context.produce_candidate(
let (block_data, _, signature) = para_context.produce_candidate(
ingress.0.iter().flat_map(|&(id, ref msgs)| msgs.iter().cloned().map(move |msg| (id, msg)))
);
......
......@@ -8,6 +8,9 @@ futures = "0.1.17"
parking_lot = "0.4"
tokio-timer = "0.1.2"
ed25519 = { path = "../../substrate/ed25519" }
error-chain = "0.11"
polkadot-api = { path = "../api" }
polkadot-collator = { path = "../collator" }
polkadot-primitives = { path = "../primitives" }
polkadot-statement-table = { path = "../statement-table" }
substrate-bft = { path = "../../substrate/bft" }
......
// Copyright 2017 Parity Technologies (UK) Ltd.
// This file is part of Polkadot.
// Polkadot is free software: you can redistribute it and/or modify
// it under the terms of the GNU General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// Polkadot is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU General Public License for more details.
// You should have received a copy of the GNU General Public License
// along with Polkadot. If not, see <http://www.gnu.org/licenses/>.
//! Errors that can occur during the consensus process.
use primitives::block::HeaderHash;
error_chain! {
links {
PolkadotApi(::polkadot_api::Error, ::polkadot_api::ErrorKind);
Bft(::bft::Error, ::bft::ErrorKind);
}
errors {
InvalidDutyRosterLength(expected: usize, got: usize) {
description("Duty Roster had invalid length"),
display("Invalid duty roster length: expected {}, got {}", expected, got),
}
ProposalNotForPolkadot {
description("Proposal provided not a Polkadot block."),
display("Proposal provided not a Polkadot block."),
}
TimestampInFuture {
description("Proposal had timestamp too far in the future."),
display("Proposal had timestamp too far in the future."),
}
WrongParentHash(expected: HeaderHash, got: HeaderHash) {
description("Proposal had wrong parent hash."),
display("Proposal had wrong parent hash. Expected {:?}, got {:?}", expected, got),
}
}
}
impl From<::bft::InputStreamConcluded> for Error {
fn from(err: ::bft::InputStreamConcluded) -> Self {
::bft::Error::from(err).into()
}
}
......@@ -29,31 +29,73 @@
//!
//! Groups themselves may be compromised by malicious authorities.
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use codec::Slicable;
use table::Table;
use table::generic::Statement as GenericStatement;
use polkadot_primitives::Hash;
use polkadot_primitives::parachain::{Id as ParaId, CandidateReceipt};
use primitives::block::Block as SubstrateBlock;
use primitives::AuthorityId;
use parking_lot::Mutex;
extern crate futures;
extern crate ed25519;
extern crate parking_lot;
extern crate tokio_timer;
extern crate polkadot_api;
extern crate polkadot_collator as collator;
extern crate polkadot_statement_table as table;
extern crate polkadot_primitives;
extern crate substrate_bft as bft;
extern crate substrate_codec as codec;
extern crate substrate_primitives as primitives;
#[macro_use]
extern crate error_chain;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use codec::Slicable;
use table::{Table, Context as TableContextTrait};
use table::generic::Statement as GenericStatement;
use polkadot_api::{PolkadotApi, BlockBuilder};
use polkadot_primitives::{Hash, Timestamp};
use polkadot_primitives::block::Block as PolkadotBlock;
use polkadot_primitives::parachain::{Id as ParaId, DutyRoster, BlockData, Extrinsic, CandidateReceipt};
use primitives::block::{Block as SubstrateBlock, Header as SubstrateHeader, HeaderHash, Id as BlockId};
use primitives::AuthorityId;
use futures::prelude::*;
use futures::future;
use parking_lot::Mutex;
pub use self::error::{ErrorKind, Error};
mod error;
/// A handle to a statement table router.
pub trait TableRouter {
/// Errors when fetching data from the network.
type Error;
/// Future that resolves when candidate data is fetched.
type FetchCandidate: IntoFuture<Item=BlockData,Error=Self::Error>;
/// Future that resolves when extrinsic candidate data is fetched.
type FetchExtrinsic: IntoFuture<Item=Extrinsic,Error=Self::Error>;
/// Note local candidate data.
fn local_candidate_data(&self, block_data: BlockData, extrinsic: Extrinsic);
/// Fetch block data for a specific candidate.
fn fetch_block_data(&self, candidate: &CandidateReceipt) -> Self::FetchCandidate;
/// Fetch extrinsic data for a specific candidate.
fn fetch_extrinsic_data(&self, candidate: &CandidateReceipt) -> Self::FetchExtrinsic;
}
/// A long-lived network which can create statement table routing instances.
pub trait Network {
/// The table router type. This should handle importing of any statements,
/// routing statements to peers, and driving completion of any `StatementProducers`.
type TableRouter: TableRouter;
/// Instantiate a table router using the given shared table.
fn table_router(&self, table: Arc<SharedTable>) -> Self::TableRouter;
}
/// Information about a specific group.
#[derive(Debug, Clone)]
#[derive(Debug, Clone, Default)]
pub struct GroupInfo {
/// Authorities meant to check validity of candidates.
pub validity_guarantors: HashSet<AuthorityId>,
......@@ -89,6 +131,10 @@ impl table::Context for TableContext {
}
impl TableContext {
fn local_id(&self) -> AuthorityId {
self.key.public().0
}
fn sign_statement(&self, statement: table::Statement) -> table::SignedStatement {
let signature = sign_table_statement(&statement, &self.key, &self.parent_hash);
let local_id = self.key.public().0;
......@@ -124,16 +170,120 @@ pub fn sign_table_statement(statement: &table::Statement, key: &ed25519::Pair, p
struct SharedTableInner {
table: Table<TableContext>,
proposed_digest: Option<Hash>,
checked_validity: HashSet<Hash>,
checked_availability: HashSet<Hash>,
}
impl SharedTableInner {
fn import_statement(
// Import a single statement. Provide a handle to a table router.
fn import_statement<R: TableRouter>(
&mut self,
context: &TableContext,
statement: ::table::SignedStatement,
router: &R,
statement: table::SignedStatement,
received_from: Option<AuthorityId>,
) -> Option<table::Summary> {
self.table.import_statement(context, statement, received_from)
) -> StatementProducer<<R::FetchCandidate as IntoFuture>::Future, <R::FetchExtrinsic as IntoFuture>::Future> {
let mut producer = StatementProducer {
fetch_block_data: None,
fetch_extrinsic: None,
produced_statements: Default::default(),
_key: context.key.clone(),
};
let summary = match self.table.import_statement(context, statement, received_from) {
Some(summary) => summary,
None => return producer,
};
let local_id = context.local_id();
let is_validity_member = context.is_member_of(&local_id, &summary.group_id);
let is_availability_member =
context.is_availability_guarantor_of(&local_id, &summary.group_id);
let digest = &summary.candidate;
// TODO: consider a strategy based on the number of candidate votes as well.
// only check validity if this wasn't locally proposed.
let checking_validity = is_validity_member
&& self.proposed_digest.as_ref().map_or(true, |d| d != digest)
&& self.checked_validity.insert(digest.clone());
let checking_availability = is_availability_member && self.checked_availability.insert(digest.clone());
if checking_validity || checking_availability {
match self.table.get_candidate(&digest) {
None => {} // TODO: handle table inconsistency somehow?
Some(candidate) => {
if checking_validity {
producer.fetch_block_data = Some(router.fetch_block_data(candidate).into_future().fuse());
}
if checking_availability {
producer.fetch_extrinsic = Some(router.fetch_extrinsic_data(candidate).into_future().fuse());
}
}
}
}
producer
}
}
/// Produced statements about a specific candidate.
/// Both may be `None`.
#[derive(Default)]
pub struct ProducedStatements {
/// A statement about the validity of the candidate.
pub validity: Option<table::Statement>,
/// A statement about the availability of the candidate.
pub availability: Option<table::Statement>,
}
/// Future that produces statements about a specific candidate.
pub struct StatementProducer<D: Future, E: Future> {
fetch_block_data: Option<future::Fuse<D>>,
fetch_extrinsic: Option<future::Fuse<E>>,
produced_statements: ProducedStatements,
_key: Arc<ed25519::Pair>,
}
impl<D, E, Err> Future for StatementProducer<D, E>
where
D: Future<Item=BlockData,Error=Err>,
E: Future<Item=Extrinsic,Error=Err>,
{
type Item = ProducedStatements;
type Error = Err;
fn poll(&mut self) -> Poll<ProducedStatements, Err> {
let mut done = true;
if let Some(ref mut fetch_block_data) = self.fetch_block_data {
match fetch_block_data.poll()? {
Async::Ready(_block_data) => {
// TODO [PoC-2] : validate block data here and make statement.
},
Async::NotReady => {
done = false;
}
}
}
if let Some(ref mut fetch_extrinsic) = self.fetch_extrinsic {
match fetch_extrinsic.poll()? {
Async::Ready(_extrinsic) => {
// TODO [PoC-2]: guarantee availability of data and make statment.
}
Async::NotReady => {
done = false;
}
}
}
if done {