fix(light): make `OnDemand` generic instead of using the concrete type (#10514)

* ethcore: add clique engine (#9981)

* fix broken sync

* correct seal fields

* ethcore: fix comment

* parity: remove duplicate params

* clique: fix whitespaces

* ethcore: fix goerli chain spec

* refactor signer_snapshot into pending/finalized state

* move close_block_extra_data after seal is applied

* refactor most of the logic into the signer_snapshot

* clique: refactor locking logic out of the consensus engine interface

* Fix jsonspec and add an unittest

* Replace space with tabs

* Unbroke sync

* Fix broken sync

* 1/2 state tracking without votes

* 2/2 implement vote tracking

* ci: use travis for goerli

* ci: setup a clique network

* ci: sync a görli node

* add clique deploy script

* ci: fix paths in clique deploy script

* ci: use docker compose

* ci: fix travis job names

* ci: fix build deps

* ci: massively reduce tests

* Revert "ci: massively reduce tests"

This reverts commit 6369f0b069ed2607a7e9f2e1d85489bacdc43384.

* ci: run cargo test directly

* ci: separate build and test stages

* ci: cache rust installation

* ci: simplify ci stages

* ci: make clique deploy script executable

* ci: shutdown goerli sync after 20min

* ci: remove slow sync stage

* ci: use timeout to finish jobs

* ci: fix build path

* ci: use absolute paths to end this confusion

* ci: add geth and parity to path

* ci: be more verbose

* ci: allow for more relaxed caching timeout

* ci: update repositories for custom ppa

* ci: fix typo in file name

* ci: fix docker compose file

* ci: add ethkey to docker

* ci: make sure deploy script is up to date with upstream

* ci: stop docker container after certain time

* ci: force superuser to update permissions on docker files

* ci: reduce run time of script to ~30 min

* ci: remove duplicate caching in travis

* remove trace statements

* clique: add more validation involving the recent signer list

* ethcore: enable constantinople for rinkeby

* ethcore: fix whitespaces in rinkeby spec

* ethcore: reformat goerli.json

* Revert "ci: remove duplicate caching in travis"

This reverts commit a562838d3d194d37f9871dcbe00b783637978f89.

* tmp commit

* another tmp commit

* it builds!

* add sealing capabilities

* add seal_header hook to allow separation of block seal/importing code paths

* clique: remove populate_from_parent.

* add panic

* make turn delay random

* initialize OpenBlock properly in 'enact'

* misc: remove duplicate lines

* misc: fix license headers

* misc: convert spaces to tabs

* misc: fix tabs

* Update Cargo.toml

* Update Cargo.toml

* Update Cargo.toml

* clique: ensure validator restores state before trying to seal

* clique: make 'state' return an Error.  Make some error messages more clear

* Fix compile error after rebase & toolchain upgrade

* fix a bunch of import warnings

* Refactor code

* Fix permissions

* Refactoring syncing

* Implement full validator checks

* Refactor util functions to seperate file

* mining 1

* ethcore: add chainspec for kotti

* ethcore: rename pre-goerli configs

* ethcore: load kotti chain spec

* cli: add kotti to params

* Implement working local sealing

* making sealing & syncing work together

* Relax timestamp checking

* ethcore: prepare for the real goerli to launch

* Implement NOTURN wiggle properly & cleanupnup warnings

* Implement vote casting

* Update docs & skip signing if no signer

* Optimize step-service interval

* Record state on local sealed block

* Fix script filemode

* Cleaning up codebase

* restore enact trace logging

* Delete clique.sh and move sync.sh

* remove travis.yml

* Remove dead code

* Cleanup compile warning

* address review comments

* adding more comments and removing unwrap()

* ci: remove sync script

* Address review comments

* fix compile error

* adding better debugging for timing

* Implement an dedicated thread for sealing timing

* fix(add helper for timestamp overflows) (#10330)

* fix(add helper timestamp overflows)

* fix(simplify code)

* fix(make helper private)

* snap: official image / test (#10168)

* official image / test

* fix / test

* bit more necromancy

* fix paths

* add source bin/df /test

* add source bin/df /test2

* something w paths /test

* something w paths /test

* add source-type /test

* show paths /test

* copy plugin /test

* plugin -> nil

* install rhash

* no questions while installing rhash

* publish snap only for release

* fix(docker): fix not receives SIGINT (#10059)

* fix(docker): fix not receives SIGINT

* fix: update with reviews

* update with review

* update

* update

* Don't add discovery initiators to the node table (#10305)

* Don't add discovery initiators to the node table

* Use enums for tracking state of the nodes in discovery

* Dont try to ping ourselves

* Fix minor nits

* Update timeouts when observing an outdated node

* Extracted update_bucket_record from update_node

* Fixed typo

* Fix two final nits from @todr

* change docker image based on debian instead of ubuntu due to the chan… (#10336)

* change docker image based on debian instead of ubuntu due to the changes of the build container

* role back docker build image and docker deploy image to ubuntu:xenial based (#10338)

* Bundle protocol and packet_id together in chain sync (#10315)

Define a new `enum` where devp2p subprotocol packet ids (currently eth and par) are defined. Additionally provide functionality to query id value and protocol of a given id object.

* snap: prefix version and populate candidate channel (#10343)

* snap: populate candidate releases with beta snaps to avoid stale channel

* snap: prefix version with v*

* addressing review comments

* engine: fix copyright header

* scripts: restore permissions on sign command

* ethcore: enforce tabs

* ethcore: enforce tabs

* ethcore: enforce tabs

* addressing comments

* addressing comments

* addressing more comments

* addressing more comments

* addressing more comments

* addressing more comments

* addressing more comments

* json-spec: fix clique epoch to non-zero u64

* ci: enable travis for parity goerli

* ci: don't separate build and test step

* ci: don't run c++ tests on travis

* ci: simplify cargo test to squeeze into travis timeout

* ci: don't run tests on travis at all

* style(fixes)

* fix(add tests)

* fix(recent_signer bug)

* fix(complete all tests)

* fix(nits)

* fix(simplify asserts)

* fix(cliqueState): simplify code

* fix(nits)

* docs(comments what's need to fixed)

* fix(revert unintended changes)

* fix(tests)

* fix(logs): voting logs

* fix(readability + more logs)

* fix(sync)

* docs(add missing licens header)

* fix(log): info! -> trace!

* docs(fix nits) + fix(remove assert)

* perf(use counter instead of vec)

* fix(remove needless block in match)

* fix(faulty comment)

* grumbles(docs for tests)

* fix(nits)

* fix(revert_vote): only remove vote when votes == 0

* fix(vote counter): checked arithmetics

* fix(simplify tests)

* fix(nits)

* fix(clique): err types

* fix(clique utils): make use of errors

* fix(cleanup nits)

* fix(clique sealing): don't read state no signer

* fix(replace Vec<Signers> with BTreeSet<Signers>)

* fix(tests): BTreeSet and more generic helpers

* fix(nits)

* fix(ethcore_block_seal): remove needless `Box`

* fix(faulty log): info -> trace

* fix(checked SystemTime): prevent SystemTime panics

* style(chain cfg): space after `:`

* style(fn enact): fix whitespace

* docs(clique): StepService

* docs(nit): fix faulty comment

* docs(fix typo)

* style(fix bad indentation)

* fix(bad regex match)

* grumble(on_seal_block): make `&mut` to avoid clone

* docs(on_seal_block): fix faulty documentation

* Delete .travis.yml

* docs: remove eth hf references in spec

* Update client.rs

* fix(nits)

* fix(clique step): `RwLock` -> `AtomicBool`

* fix(clique): use `Duration::as_millis`

* Clean up some Clique documentation

* Add trace information to eth_estimateGas (#10519)

* Add trace information to eth_estimateGas

* replace unwrap better version

* change vm::Error formatter to more user-friendly

* remove extra error format

* use map_or instead sequence of map/unwrap_or

* fix(light/on_demand): extract as a trait

* fix(grumble): OnDemand remove needless trait bound
This commit is contained in:
Niklas Adolfsson 2019-03-27 14:46:20 +01:00 committed by soc1c
parent 7d26a82232
commit 407de5e8c4
9 changed files with 181 additions and 134 deletions

View File

@ -94,6 +94,24 @@ pub mod error {
}
}
/// Public interface for performing network requests `OnDemand`
pub trait OnDemandRequester: Send + Sync {
/// Submit a strongly-typed batch of requests.
///
/// Fails if back-reference are not coherent.
fn request<T>(&self, ctx: &BasicContext, requests: T) -> Result<OnResponses<T>, basic_request::NoSuchOutput>
where
T: request::RequestAdapter;
/// Submit a vector of requests to be processed together.
///
/// Fails if back-references are not coherent.
/// The returned vector of responses will correspond to the requests exactly.
fn request_raw(&self, ctx: &BasicContext, requests: Vec<Request>)
-> Result<Receiver<PendingResponse>, basic_request::NoSuchOutput>;
}
// relevant peer info.
#[derive(Debug, Clone, PartialEq, Eq)]
struct Peer {
@ -355,6 +373,74 @@ pub struct OnDemand {
request_number_of_consecutive_errors: usize
}
impl OnDemandRequester for OnDemand {
fn request_raw(&self, ctx: &BasicContext, requests: Vec<Request>)
-> Result<Receiver<PendingResponse>, basic_request::NoSuchOutput>
{
let (sender, receiver) = oneshot::channel();
if requests.is_empty() {
assert!(sender.send(Ok(Vec::new())).is_ok(), "receiver still in scope; qed");
return Ok(receiver);
}
let mut builder = basic_request::Builder::default();
let responses = Vec::with_capacity(requests.len());
let mut header_producers = HashMap::new();
for (i, request) in requests.into_iter().enumerate() {
let request = CheckedRequest::from(request);
// ensure that all requests needing headers will get them.
if let Some((idx, field)) = request.needs_header() {
// a request chain with a header back-reference is valid only if it both
// points to a request that returns a header and has the same back-reference
// for the block hash.
match header_producers.get(&idx) {
Some(ref f) if &field == *f => {}
_ => return Err(basic_request::NoSuchOutput),
}
}
if let CheckedRequest::HeaderByHash(ref req, _) = request {
header_producers.insert(i, req.0);
}
builder.push(request)?;
}
let requests = builder.build();
let net_requests = requests.clone().map_requests(|req| req.into_net_request());
let capabilities = guess_capabilities(requests.requests());
self.submit_pending(ctx, Pending {
requests,
net_requests,
required_capabilities: capabilities,
responses,
sender,
request_guard: RequestGuard::new(
self.request_number_of_consecutive_errors as u32,
self.request_backoff_rounds_max,
self.request_backoff_start,
self.request_backoff_max,
),
response_guard: ResponseGuard::new(self.response_time_window),
});
Ok(receiver)
}
fn request<T>(&self, ctx: &BasicContext, requests: T) -> Result<OnResponses<T>, basic_request::NoSuchOutput>
where T: request::RequestAdapter
{
self.request_raw(ctx, requests.make_requests()).map(|recv| OnResponses {
receiver: recv,
_marker: PhantomData,
})
}
}
impl OnDemand {
/// Create a new `OnDemand` service with the given cache.
@ -415,77 +501,6 @@ impl OnDemand {
me
}
/// Submit a vector of requests to be processed together.
///
/// Fails if back-references are not coherent.
/// The returned vector of responses will correspond to the requests exactly.
pub fn request_raw(&self, ctx: &BasicContext, requests: Vec<Request>)
-> Result<Receiver<PendingResponse>, basic_request::NoSuchOutput>
{
let (sender, receiver) = oneshot::channel();
if requests.is_empty() {
assert!(sender.send(Ok(Vec::new())).is_ok(), "receiver still in scope; qed");
return Ok(receiver);
}
let mut builder = basic_request::Builder::default();
let responses = Vec::with_capacity(requests.len());
let mut header_producers = HashMap::new();
for (i, request) in requests.into_iter().enumerate() {
let request = CheckedRequest::from(request);
// ensure that all requests needing headers will get them.
if let Some((idx, field)) = request.needs_header() {
// a request chain with a header back-reference is valid only if it both
// points to a request that returns a header and has the same back-reference
// for the block hash.
match header_producers.get(&idx) {
Some(ref f) if &field == *f => {}
_ => return Err(basic_request::NoSuchOutput),
}
}
if let CheckedRequest::HeaderByHash(ref req, _) = request {
header_producers.insert(i, req.0);
}
builder.push(request)?;
}
let requests = builder.build();
let net_requests = requests.clone().map_requests(|req| req.into_net_request());
let capabilities = guess_capabilities(requests.requests());
self.submit_pending(ctx, Pending {
requests,
net_requests,
required_capabilities: capabilities,
responses,
sender,
request_guard: RequestGuard::new(
self.request_number_of_consecutive_errors as u32,
self.request_backoff_rounds_max,
self.request_backoff_start,
self.request_backoff_max,
),
response_guard: ResponseGuard::new(self.response_time_window),
});
Ok(receiver)
}
/// Submit a strongly-typed batch of requests.
///
/// Fails if back-reference are not coherent.
pub fn request<T>(&self, ctx: &BasicContext, requests: T) -> Result<OnResponses<T>, basic_request::NoSuchOutput>
where T: request::RequestAdapter
{
self.request_raw(ctx, requests.make_requests()).map(|recv| OnResponses {
receiver: recv,
_marker: PhantomData,
})
}
// maybe dispatch pending requests.
// sometimes

View File

@ -29,7 +29,7 @@ use std::sync::Arc;
use std::time::{Duration, Instant};
use std::thread;
use super::{request, OnDemand, Peer, HeaderRef};
use super::{request, OnDemand, OnDemandRequester, Peer, HeaderRef};
// useful contexts to give the service.
enum Context {

View File

@ -27,7 +27,7 @@ use futures::{future, Future};
use futures::future::Either;
use light::client::fetch::ChainDataFetcher;
use light::on_demand::{request, OnDemand};
use light::on_demand::{request, OnDemand, OnDemandRequester};
use parking_lot::RwLock;
use ethereum_types::H256;

View File

@ -24,7 +24,7 @@ use sync::{LightSync, LightNetworkDispatcher};
use io::{IoContext, IoHandler, TimerToken};
use light::client::LightChainClient;
use light::on_demand::{request, OnDemand};
use light::on_demand::{request, OnDemand, OnDemandRequester};
use light::TransactionQueue;
use futures::{future, Future};

View File

@ -20,7 +20,7 @@ use ethereum_types::{H256, Address, U256};
use light::TransactionQueue as LightTransactionQueue;
use light::cache::Cache as LightDataCache;
use light::client::LightChainClient;
use light::on_demand::{request, OnDemand};
use light::on_demand::{request, OnDemandRequester};
use parking_lot::{Mutex, RwLock};
use stats::Corpus;
use sync::{LightSyncProvider, LightNetworkDispatcher, ManageNetwork};
@ -37,13 +37,17 @@ use v1::types::{RichRawTransaction as RpcRichRawTransaction,};
use super::{Dispatcher, Accounts, SignWith, PostSign};
/// Dispatcher for light clients -- fetches default gas price, next nonce, etc. from network.
pub struct LightDispatcher<S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static> {
pub struct LightDispatcher<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
/// Sync service.
pub sync: Arc<S>,
/// Header chain client.
pub client: Arc<LightChainClient>,
/// On-demand request service.
pub on_demand: Arc<OnDemand>,
pub on_demand: Arc<OD>,
/// Data cache.
pub cache: Arc<Mutex<LightDataCache>>,
/// Transaction queue.
@ -54,9 +58,10 @@ pub struct LightDispatcher<S: LightSyncProvider + LightNetworkDispatcher + Manag
pub gas_price_percentile: usize,
}
impl<S> LightDispatcher<S>
impl<S, OD> LightDispatcher<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
/// Create a new `LightDispatcher` from its requisite parts.
///
@ -64,7 +69,7 @@ where
pub fn new(
sync: Arc<S>,
client: Arc<LightChainClient>,
on_demand: Arc<OnDemand>,
on_demand: Arc<OD>,
cache: Arc<Mutex<LightDataCache>>,
transaction_queue: Arc<RwLock<LightTransactionQueue>>,
nonces: Arc<Mutex<nonce::Reservations>>,
@ -117,9 +122,10 @@ where
}
}
impl<S> Clone for LightDispatcher<S>
impl<S, OD> Clone for LightDispatcher<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
fn clone(&self) -> Self {
Self {
@ -134,9 +140,10 @@ where
}
}
impl<S> Dispatcher for LightDispatcher<S>
impl<S, OD> Dispatcher for LightDispatcher<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
// Ignore the `force_nonce` flag in order to always query the network when fetching the nonce and
// the account state. If the nonce is specified in the transaction use that nonce instead but do the
@ -239,12 +246,16 @@ where
/// Get a recent gas price corpus.
// TODO: this could be `impl Trait`.
pub fn fetch_gas_price_corpus<S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static>(
pub fn fetch_gas_price_corpus<S, OD>(
sync: Arc<S>,
client: Arc<LightChainClient>,
on_demand: Arc<OnDemand>,
on_demand: Arc<OD>,
cache: Arc<Mutex<LightDataCache>>,
) -> BoxFuture<Corpus<U256>> {
) -> BoxFuture<Corpus<U256>>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
const GAS_PRICE_SAMPLE_SIZE: usize = 100;
if let Some(cached) = { cache.lock().gas_price_corpus() } {

View File

@ -35,16 +35,15 @@ use light::cache::Cache;
use light::client::LightChainClient;
use light::{cht, MAX_HEADERS_PER_REQUEST};
use light::on_demand::{
request, OnDemand, HeaderRef, Request as OnDemandRequest,
request, OnDemandRequester, HeaderRef, Request as OnDemandRequest,
Response as OnDemandResponse, ExecutionResult,
};
use light::on_demand::error::Error as OnDemandError;
use light::request::Field;
use sync::{LightNetworkDispatcher, ManageNetwork, LightSyncProvider};
use ethereum_types::{U256, Address};
use ethereum_types::Address;
use hash::H256;
use parking_lot::Mutex;
use fastmap::H256FastMap;
@ -57,9 +56,10 @@ use v1::types::{BlockNumber, CallRequest, Log, Transaction};
const NO_INVALID_BACK_REFS_PROOF: &str = "Fails only on invalid back-references; back-references here known to be valid; qed";
const WRONG_RESPONSE_AMOUNT_TYPE_PROOF: &str = "responses correspond directly with requests in amount and type; qed";
pub fn light_all_transactions<S>(dispatch: &Arc<dispatch::LightDispatcher<S>>) -> impl Iterator<Item=PendingTransaction>
pub fn light_all_transactions<S, OD>(dispatch: &Arc<dispatch::LightDispatcher<S, OD>>) -> impl Iterator<Item=PendingTransaction>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
let txq = dispatch.transaction_queue.read();
let chain_info = dispatch.client.chain_info();
@ -71,12 +71,15 @@ where
/// Helper for fetching blockchain data either from the light client or the network
/// as necessary.
pub struct LightFetch<S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static>
pub struct LightFetch<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
/// The light client.
pub client: Arc<LightChainClient>,
/// The on-demand request service.
pub on_demand: Arc<OnDemand>,
pub on_demand: Arc<OD>,
/// Handle to the network.
pub sync: Arc<S>,
/// The light data cache.
@ -85,9 +88,10 @@ pub struct LightFetch<S: LightSyncProvider + LightNetworkDispatcher + ManageNetw
pub gas_price_percentile: usize,
}
impl<S> Clone for LightFetch<S>
impl<S, OD> Clone for LightFetch<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
fn clone(&self) -> Self {
Self {
@ -136,9 +140,10 @@ fn extract_header(res: &[OnDemandResponse], header: HeaderRef) -> Option<encoded
}
}
impl<S> LightFetch<S>
impl<S, OD> LightFetch<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
// push the necessary requests onto the request chain to get the header by the given ID.
// yield a header reference which other requests can use.
@ -277,7 +282,7 @@ where
action: req.to.map_or(Action::Create, Action::Call),
gas: req.gas.unwrap_or_else(|| START_GAS.into()),
gas_price,
value: req.value.unwrap_or_else(U256::zero),
value: req.value.unwrap_or_default(),
data: req.data.unwrap_or_default(),
}))
)
@ -387,7 +392,7 @@ where
block_index += 1;
}
}
future::ok::<_,OnDemandError>(matches)
future::ok::<_, OnDemandError>(matches)
})
.map_err(errors::on_demand_error)
.map(|matches| matches.into_iter().map(|(_, v)| v).collect())
@ -657,22 +662,24 @@ where
}
}
struct ExecuteParams<S>
struct ExecuteParams<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
from: Address,
tx: EthTransaction,
hdr: encoded::Header,
env_info: ::vm::EnvInfo,
engine: Arc<::ethcore::engines::EthEngine>,
on_demand: Arc<OnDemand>,
on_demand: Arc<OD>,
sync: Arc<S>,
}
impl<S> Clone for ExecuteParams<S>
impl<S, OD> Clone for ExecuteParams<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
fn clone(&self) -> Self {
Self {
@ -689,9 +696,10 @@ where
// Has a peer execute the transaction with given params. If `gas_known` is false, this will set the `gas value` to the
// `required gas value` unless it exceeds the block gas limit
fn execute_read_only_tx<S>(gas_known: bool, params: ExecuteParams<S>) -> impl Future<Item = ExecutionResult, Error = Error> + Send
fn execute_read_only_tx<S, OD>(gas_known: bool, params: ExecuteParams<S, OD>) -> impl Future<Item = ExecutionResult, Error = Error> + Send
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
if !gas_known {
Box::new(future::loop_fn(params, |mut params| {

View File

@ -33,7 +33,7 @@ use ethcore::client::{BlockChainClient, ChainNotify, NewBlocks, ChainRouteType,
use ethereum_types::H256;
use light::cache::Cache;
use light::client::{LightChainClient, LightChainNotify};
use light::on_demand::OnDemand;
use light::on_demand::OnDemandRequester;
use parity_runtime::Executor;
use parking_lot::{RwLock, Mutex};
@ -89,14 +89,15 @@ impl<C> EthPubSubClient<C> {
}
}
impl<S> EthPubSubClient<LightFetch<S>>
impl<S, OD> EthPubSubClient<LightFetch<S, OD>>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
/// Creates a new `EthPubSubClient` for `LightClient`.
pub fn light(
client: Arc<LightChainClient>,
on_demand: Arc<OnDemand>,
on_demand: Arc<OD>,
sync: Arc<S>,
cache: Arc<Mutex<Cache>>,
executor: Executor,
@ -194,9 +195,10 @@ pub trait LightClient: Send + Sync {
fn logs(&self, filter: EthFilter) -> BoxFuture<Vec<Log>>;
}
impl<S> LightClient for LightFetch<S>
impl<S, OD> LightClient for LightFetch<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
fn block_header(&self, id: BlockId) -> Option<encoded::Header> {
self.client.block_header(id)

View File

@ -26,7 +26,7 @@ use jsonrpc_core::futures::future::Either;
use light::cache::Cache as LightDataCache;
use light::client::LightChainClient;
use light::{cht, TransactionQueue};
use light::on_demand::{request, OnDemand};
use light::on_demand::{request, OnDemandRequester};
use ethereum_types::{Address, H64, H160, H256, U64, U256};
use hash::{KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP};
@ -54,10 +54,10 @@ use sync::{LightSyncInfo, LightSyncProvider, LightNetworkDispatcher, ManageNetwo
const NO_INVALID_BACK_REFS: &str = "Fails only on invalid back-references; back-references here known to be valid; qed";
/// Light client `ETH` (and filter) RPC.
pub struct EthClient<C, S: LightSyncProvider + LightNetworkDispatcher + 'static> {
pub struct EthClient<C, S: LightSyncProvider + LightNetworkDispatcher + 'static, OD: OnDemandRequester + 'static> {
sync: Arc<S>,
client: Arc<C>,
on_demand: Arc<OnDemand>,
on_demand: Arc<OD>,
transaction_queue: Arc<RwLock<TransactionQueue>>,
accounts: Arc<Fn() -> Vec<Address> + Send + Sync>,
cache: Arc<Mutex<LightDataCache>>,
@ -67,9 +67,10 @@ pub struct EthClient<C, S: LightSyncProvider + LightNetworkDispatcher + 'static>
deprecation_notice: DeprecationNotice,
}
impl<C, S> Clone for EthClient<C, S>
impl<C, S, OD> Clone for EthClient<C, S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + 'static
S: LightSyncProvider + LightNetworkDispatcher + 'static,
OD: OnDemandRequester + 'static
{
fn clone(&self) -> Self {
// each instance should have its own poll manager.
@ -88,17 +89,18 @@ where
}
}
impl<C, S> EthClient<C, S>
impl<C, S, OD> EthClient<C, S, OD>
where
C: LightChainClient + 'static,
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
/// Create a new `EthClient` with a handle to the light sync instance, client,
/// and on-demand request service, which is assumed to be attached as a handler.
pub fn new(
sync: Arc<S>,
client: Arc<C>,
on_demand: Arc<OnDemand>,
on_demand: Arc<OD>,
transaction_queue: Arc<RwLock<TransactionQueue>>,
accounts: Arc<Fn() -> Vec<Address> + Send + Sync>,
cache: Arc<Mutex<LightDataCache>>,
@ -120,7 +122,7 @@ where
}
/// Create a light data fetcher instance.
fn fetcher(&self) -> LightFetch<S>
fn fetcher(&self) -> LightFetch<S, OD>
{
LightFetch {
client: self.client.clone(),
@ -218,10 +220,11 @@ where
}
}
impl<C, S> Eth for EthClient<C, S>
impl<C, S, OD> Eth for EthClient<C, S, OD>
where
C: LightChainClient + 'static,
S: LightSyncInfo + LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncInfo + LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
type Metadata = Metadata;
@ -533,10 +536,11 @@ where
}
// This trait implementation triggers a blanked impl of `EthFilter`.
impl<C, S> Filterable for EthClient<C, S>
impl<C, S, OD> Filterable for EthClient<C, S, OD>
where
C: LightChainClient + 'static,
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
fn best_block_number(&self) -> u64 { self.client.chain_info().best_block_number }

View File

@ -30,6 +30,7 @@ use ethcore_logger::RotatingLogger;
use jsonrpc_core::{Result, BoxFuture};
use jsonrpc_core::futures::{future, Future};
use light::on_demand::OnDemandRequester;
use v1::helpers::{self, errors, ipfs, NetworkSettings, verify_signature};
use v1::helpers::external_signer::{SignerService, SigningQueue};
use v1::helpers::dispatch::LightDispatcher;
@ -48,8 +49,12 @@ use v1::types::{
use Host;
/// Parity implementation for light client.
pub struct ParityClient<S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static> {
light_dispatch: Arc<LightDispatcher<S>>,
pub struct ParityClient<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
light_dispatch: Arc<LightDispatcher<S, OD>>,
logger: Arc<RotatingLogger>,
settings: Arc<NetworkSettings>,
signer: Option<Arc<SignerService>>,
@ -57,13 +62,14 @@ pub struct ParityClient<S: LightSyncProvider + LightNetworkDispatcher + ManageNe
gas_price_percentile: usize,
}
impl<S> ParityClient<S>
impl<S, OD> ParityClient<S, OD>
where
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
/// Creates new `ParityClient`.
pub fn new(
light_dispatch: Arc<LightDispatcher<S>>,
light_dispatch: Arc<LightDispatcher<S, OD>>,
logger: Arc<RotatingLogger>,
settings: Arc<NetworkSettings>,
signer: Option<Arc<SignerService>>,
@ -81,7 +87,7 @@ where
}
/// Create a light blockchain data fetcher.
fn fetcher(&self) -> LightFetch<S>
fn fetcher(&self) -> LightFetch<S, OD>
{
LightFetch {
client: self.light_dispatch.client.clone(),
@ -93,9 +99,10 @@ where
}
}
impl<S> Parity for ParityClient<S>
impl<S, OD> Parity for ParityClient<S, OD>
where
S: LightSyncInfo + LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static
S: LightSyncInfo + LightSyncProvider + LightNetworkDispatcher + ManageNetwork + 'static,
OD: OnDemandRequester + 'static
{
type Metadata = Metadata;