Light Client: fetch transactions/receipts by transaction hash (#6641)

* rpc: transaction/receipt requests made async

* rpc: light client fetches transaction and uncle by hash/index

* on_demand: request type for transaction index

* serve transaction index requests in light protocol

* add a test for transaction index serving

* fetch transaction and receipts by hash on light client

* fix decoding tests

* light: more lenient cost table parsing (backwards compatible)

* fix tests and warnings

* LES -> PIP

* Update provider.rs

* proper doc comments for public functions
This commit is contained in:
Robert Habermeier 2017-10-08 18:19:27 +02:00 committed by Gav Wood
parent 360ecd3728
commit b010fb5004
15 changed files with 462 additions and 114 deletions

View File

@ -430,7 +430,11 @@ impl LightProtocol {
// compute and deduct cost.
let pre_creds = creds.current();
let cost = params.compute_cost_multi(requests.requests());
let cost = match params.compute_cost_multi(requests.requests()) {
Some(cost) => cost,
None => return Err(Error::NotServer),
};
creds.deduct_cost(cost)?;
trace!(target: "pip", "requesting from peer {}. Cost: {}; Available: {}",
@ -924,7 +928,7 @@ impl LightProtocol {
peer.local_credits.deduct_cost(peer.local_flow.base_cost())?;
for request_rlp in raw.at(1)?.iter().take(MAX_REQUESTS) {
let request: Request = request_rlp.as_val()?;
let cost = peer.local_flow.compute_cost(&request);
let cost = peer.local_flow.compute_cost(&request).ok_or(Error::NotServer)?;
peer.local_credits.deduct_cost(cost)?;
request_builder.push(request).map_err(|_| Error::BadBackReference)?;
}
@ -939,7 +943,7 @@ impl LightProtocol {
match complete_req {
CompleteRequest::Headers(req) => self.provider.block_headers(req).map(Response::Headers),
CompleteRequest::HeaderProof(req) => self.provider.header_proof(req).map(Response::HeaderProof),
CompleteRequest::TransactionIndex(_) => None, // don't answer these yet, but leave them in protocol.
CompleteRequest::TransactionIndex(req) => self.provider.transaction_index(req).map(Response::TransactionIndex),
CompleteRequest::Body(req) => self.provider.block_body(req).map(Response::Body),
CompleteRequest::Receipts(req) => self.provider.block_receipts(req).map(Response::Receipts),
CompleteRequest::Account(req) => self.provider.account_proof(req).map(Response::Account),

View File

@ -79,19 +79,42 @@ impl Credits {
}
/// A cost table, mapping requests to base and per-request costs.
/// Costs themselves may be missing.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct CostTable {
base: U256, // cost per packet.
headers: U256, // cost per header
transaction_index: U256,
body: U256,
receipts: U256,
account: U256,
storage: U256,
code: U256,
header_proof: U256,
transaction_proof: U256, // cost per gas.
epoch_signal: U256,
headers: Option<U256>, // cost per header
transaction_index: Option<U256>,
body: Option<U256>,
receipts: Option<U256>,
account: Option<U256>,
storage: Option<U256>,
code: Option<U256>,
header_proof: Option<U256>,
transaction_proof: Option<U256>, // cost per gas.
epoch_signal: Option<U256>,
}
impl CostTable {
fn costs_set(&self) -> usize {
let mut num_set = 0;
{
let mut incr_if_set = |cost: &Option<_>| if cost.is_some() { num_set += 1 };
incr_if_set(&self.headers);
incr_if_set(&self.transaction_index);
incr_if_set(&self.body);
incr_if_set(&self.receipts);
incr_if_set(&self.account);
incr_if_set(&self.storage);
incr_if_set(&self.code);
incr_if_set(&self.header_proof);
incr_if_set(&self.transaction_proof);
incr_if_set(&self.epoch_signal);
}
num_set
}
}
impl Default for CostTable {
@ -99,31 +122,32 @@ impl Default for CostTable {
// arbitrarily chosen constants.
CostTable {
base: 100000.into(),
headers: 10000.into(),
transaction_index: 10000.into(),
body: 15000.into(),
receipts: 5000.into(),
account: 25000.into(),
storage: 25000.into(),
code: 20000.into(),
header_proof: 15000.into(),
transaction_proof: 2.into(),
epoch_signal: 10000.into(),
headers: Some(10000.into()),
transaction_index: Some(10000.into()),
body: Some(15000.into()),
receipts: Some(5000.into()),
account: Some(25000.into()),
storage: Some(25000.into()),
code: Some(20000.into()),
header_proof: Some(15000.into()),
transaction_proof: Some(2.into()),
epoch_signal: Some(10000.into()),
}
}
}
impl Encodable for CostTable {
fn rlp_append(&self, s: &mut RlpStream) {
fn append_cost(s: &mut RlpStream, cost: &U256, kind: request::Kind) {
s.begin_list(2);
// hack around https://github.com/paritytech/parity/issues/4356
Encodable::rlp_append(&kind, s);
s.append(cost);
fn append_cost(s: &mut RlpStream, cost: &Option<U256>, kind: request::Kind) {
if let Some(ref cost) = *cost {
s.begin_list(2);
// hack around https://github.com/paritytech/parity/issues/4356
Encodable::rlp_append(&kind, s);
s.append(cost);
}
}
s.begin_list(11).append(&self.base);
s.begin_list(1 + self.costs_set()).append(&self.base);
append_cost(s, &self.headers, request::Kind::Headers);
append_cost(s, &self.transaction_index, request::Kind::TransactionIndex);
append_cost(s, &self.body, request::Kind::Body);
@ -168,21 +192,25 @@ impl Decodable for CostTable {
}
}
let unwrap_cost = |cost: Option<U256>| cost.ok_or(DecoderError::Custom("Not all costs specified in cost table."));
Ok(CostTable {
let table = CostTable {
base: base,
headers: unwrap_cost(headers)?,
transaction_index: unwrap_cost(transaction_index)?,
body: unwrap_cost(body)?,
receipts: unwrap_cost(receipts)?,
account: unwrap_cost(account)?,
storage: unwrap_cost(storage)?,
code: unwrap_cost(code)?,
header_proof: unwrap_cost(header_proof)?,
transaction_proof: unwrap_cost(transaction_proof)?,
epoch_signal: unwrap_cost(epoch_signal)?,
})
headers: headers,
transaction_index: transaction_index,
body: body,
receipts: receipts,
account: account,
storage: storage,
code: code,
header_proof: header_proof,
transaction_proof: transaction_proof,
epoch_signal: epoch_signal,
};
if table.costs_set() == 0 {
Err(DecoderError::Custom("no cost types set."))
} else {
Ok(table)
}
}
}
@ -230,7 +258,7 @@ impl FlowParams {
let serve_per_second = serve_per_second.max(1.0 / 10_000.0);
// as a percentage of the recharge per second.
U256::from((recharge as f64 / serve_per_second) as u64)
Some(U256::from((recharge as f64 / serve_per_second) as u64))
};
let costs = CostTable {
@ -256,12 +284,12 @@ impl FlowParams {
/// Create effectively infinite flow params.
pub fn free() -> Self {
let free_cost: U256 = 0.into();
let free_cost: Option<U256> = Some(0.into());
FlowParams {
limit: (!0u64).into(),
recharge: 1.into(),
costs: CostTable {
base: free_cost.clone(),
base: 0.into(),
headers: free_cost.clone(),
transaction_index: free_cost.clone(),
body: free_cost.clone(),
@ -290,9 +318,9 @@ impl FlowParams {
/// Compute the actual cost of a request, given the kind of request
/// and number of requests made.
pub fn compute_cost(&self, request: &Request) -> U256 {
pub fn compute_cost(&self, request: &Request) -> Option<U256> {
match *request {
Request::Headers(ref req) => self.costs.headers * req.max.into(),
Request::Headers(ref req) => self.costs.headers.map(|c| c * req.max.into()),
Request::HeaderProof(_) => self.costs.header_proof,
Request::TransactionIndex(_) => self.costs.transaction_index,
Request::Body(_) => self.costs.body,
@ -300,15 +328,23 @@ impl FlowParams {
Request::Account(_) => self.costs.account,
Request::Storage(_) => self.costs.storage,
Request::Code(_) => self.costs.code,
Request::Execution(ref req) => self.costs.transaction_proof * req.gas,
Request::Execution(ref req) => self.costs.transaction_proof.map(|c| c * req.gas),
Request::Signal(_) => self.costs.epoch_signal,
}
}
/// Compute the cost of a set of requests.
/// This is the base cost plus the cost of each individual request.
pub fn compute_cost_multi(&self, requests: &[Request]) -> U256 {
requests.iter().fold(self.costs.base, |cost, req| cost + self.compute_cost(req))
pub fn compute_cost_multi(&self, requests: &[Request]) -> Option<U256> {
let mut cost = self.costs.base;
for request in requests {
match self.compute_cost(request) {
Some(c) => cost = cost + c,
None => return None,
}
}
Some(cost)
}
/// Create initial credits.
@ -408,6 +444,6 @@ mod tests {
);
assert_eq!(flow_params2.costs, flow_params3.costs);
assert_eq!(flow_params.costs.headers, flow_params2.costs.headers * 2.into());
assert_eq!(flow_params.costs.headers.unwrap(), flow_params2.costs.headers.unwrap() * 2.into());
}
}

View File

@ -116,6 +116,16 @@ impl Provider for TestProvider {
self.0.client.block_header(id)
}
fn transaction_index(&self, req: request::CompleteTransactionIndexRequest)
-> Option<request::TransactionIndexResponse>
{
Some(request::TransactionIndexResponse {
num: 100,
hash: req.hash,
index: 55,
})
}
fn block_body(&self, req: request::CompleteBodyRequest) -> Option<request::BodyResponse> {
self.0.client.block_body(req)
}
@ -308,7 +318,7 @@ fn get_block_headers() {
let headers: Vec<_> = (0..10).map(|i| provider.client.block_header(BlockId::Number(i + 1)).unwrap()).collect();
assert_eq!(headers.len(), 10);
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests());
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests()).unwrap();
let response = vec![Response::Headers(HeadersResponse {
headers: headers,
@ -361,7 +371,7 @@ fn get_block_bodies() {
let request_body = make_packet(req_id, &requests);
let response = {
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests());
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests()).unwrap();
let mut response_stream = RlpStream::new_list(3);
response_stream.append(&req_id).append(&new_creds).append_list(&bodies);
@ -416,7 +426,7 @@ fn get_block_receipts() {
let response = {
assert_eq!(receipts.len(), 10);
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests());
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests()).unwrap();
let mut response_stream = RlpStream::new_list(3);
response_stream.append(&req_id).append(&new_creds).append_list(&receipts);
@ -475,7 +485,7 @@ fn get_state_proofs() {
}).unwrap()),
];
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests());
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests()).unwrap();
let mut response_stream = RlpStream::new_list(3);
response_stream.append(&req_id).append(&new_creds).append_list(&responses);
@ -517,7 +527,7 @@ fn get_contract_code() {
code: key1.iter().chain(key2.iter()).cloned().collect(),
})];
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests());
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests()).unwrap();
let mut response_stream = RlpStream::new_list(3);
@ -558,9 +568,8 @@ fn epoch_signal() {
})];
let limit = *flow_params.limit();
let cost = flow_params.compute_cost_multi(requests.requests());
let cost = flow_params.compute_cost_multi(requests.requests()).unwrap();
println!("limit = {}, cost = {}", limit, cost);
let new_creds = limit - cost;
let mut response_stream = RlpStream::new_list(3);
@ -605,9 +614,8 @@ fn proof_of_execution() {
let response = {
let limit = *flow_params.limit();
let cost = flow_params.compute_cost_multi(requests.requests());
let cost = flow_params.compute_cost_multi(requests.requests()).unwrap();
println!("limit = {}, cost = {}", limit, cost);
let new_creds = limit - cost;
let mut response_stream = RlpStream::new_list(3);
@ -713,3 +721,46 @@ fn id_guard() {
assert_eq!(peer_info.failed_requests, &[req_id_1]);
}
}
#[test]
fn get_transaction_index() {
let capabilities = capabilities();
let (provider, proto) = setup(capabilities.clone());
let flow_params = proto.flow_params.read().clone();
let cur_status = status(provider.client.chain_info());
{
let packet_body = write_handshake(&cur_status, &capabilities, &proto);
proto.on_connect(&1, &Expect::Send(1, packet::STATUS, packet_body.clone()));
proto.handle_packet(&Expect::Nothing, &1, packet::STATUS, &packet_body);
}
let req_id = 112;
let key1: H256 = U256::from(11223344).into();
let request = Request::TransactionIndex(IncompleteTransactionIndexRequest {
hash: key1.into(),
});
let requests = encode_single(request.clone());
let request_body = make_packet(req_id, &requests);
let response = {
let response = vec![Response::TransactionIndex(TransactionIndexResponse {
num: 100,
hash: key1,
index: 55,
})];
let new_creds = *flow_params.limit() - flow_params.compute_cost_multi(requests.requests()).unwrap();
let mut response_stream = RlpStream::new_list(3);
response_stream.append(&req_id).append(&new_creds).append_list(&response);
response_stream.out()
};
let expected = Expect::Respond(packet::RESPONSE, response);
proto.handle_packet(&expected, &1, packet::REQUEST, &request_body);
}

View File

@ -195,6 +195,7 @@ fn guess_capabilities(requests: &[CheckedRequest]) -> Capabilities {
caps.serve_headers = true,
CheckedRequest::HeaderByHash(_, _) =>
caps.serve_headers = true,
CheckedRequest::TransactionIndex(_, _) => {} // hashes yield no info.
CheckedRequest::Signal(_, _) =>
caps.serve_headers = true,
CheckedRequest::Body(ref req, _) => if let Ok(ref hdr) = req.0.as_ref() {

View File

@ -48,6 +48,8 @@ pub enum Request {
HeaderProof(HeaderProof),
/// A request for a header by hash.
HeaderByHash(HeaderByHash),
/// A request for the index of a transaction.
TransactionIndex(TransactionIndex),
/// A request for block receipts.
Receipts(BlockReceipts),
/// A request for a block body.
@ -135,6 +137,7 @@ macro_rules! impl_single {
// implement traits for each kind of request.
impl_single!(HeaderProof, HeaderProof, (H256, U256));
impl_single!(HeaderByHash, HeaderByHash, encoded::Header);
impl_single!(TransactionIndex, TransactionIndex, net_request::TransactionIndexResponse);
impl_single!(Receipts, BlockReceipts, Vec<Receipt>);
impl_single!(Body, Body, encoded::Block);
impl_single!(Account, Account, Option<BasicAccount>);
@ -244,6 +247,7 @@ impl From<encoded::Header> for HeaderRef {
pub enum CheckedRequest {
HeaderProof(HeaderProof, net_request::IncompleteHeaderProofRequest),
HeaderByHash(HeaderByHash, net_request::IncompleteHeadersRequest),
TransactionIndex(TransactionIndex, net_request::IncompleteTransactionIndexRequest),
Receipts(BlockReceipts, net_request::IncompleteReceiptsRequest),
Body(Body, net_request::IncompleteBodyRequest),
Account(Account, net_request::IncompleteAccountRequest),
@ -270,6 +274,12 @@ impl From<Request> for CheckedRequest {
};
CheckedRequest::HeaderProof(req, net_req)
}
Request::TransactionIndex(req) => {
let net_req = net_request::IncompleteTransactionIndexRequest {
hash: req.0.clone(),
};
CheckedRequest::TransactionIndex(req, net_req)
}
Request::Body(req) => {
let net_req = net_request::IncompleteBodyRequest {
hash: req.0.field(),
@ -326,6 +336,7 @@ impl CheckedRequest {
match self {
CheckedRequest::HeaderProof(_, req) => NetRequest::HeaderProof(req),
CheckedRequest::HeaderByHash(_, req) => NetRequest::Headers(req),
CheckedRequest::TransactionIndex(_, req) => NetRequest::TransactionIndex(req),
CheckedRequest::Receipts(_, req) => NetRequest::Receipts(req),
CheckedRequest::Body(_, req) => NetRequest::Body(req),
CheckedRequest::Account(_, req) => NetRequest::Account(req),
@ -454,6 +465,7 @@ macro_rules! match_me {
match $me {
CheckedRequest::HeaderProof($check, $req) => $e,
CheckedRequest::HeaderByHash($check, $req) => $e,
CheckedRequest::TransactionIndex($check, $req) => $e,
CheckedRequest::Receipts($check, $req) => $e,
CheckedRequest::Body($check, $req) => $e,
CheckedRequest::Account($check, $req) => $e,
@ -482,6 +494,7 @@ impl IncompleteRequest for CheckedRequest {
_ => Ok(()),
}
}
CheckedRequest::TransactionIndex(_, ref req) => req.check_outputs(f),
CheckedRequest::Receipts(_, ref req) => req.check_outputs(f),
CheckedRequest::Body(_, ref req) => req.check_outputs(f),
CheckedRequest::Account(_, ref req) => req.check_outputs(f),
@ -503,6 +516,7 @@ impl IncompleteRequest for CheckedRequest {
match self {
CheckedRequest::HeaderProof(_, req) => req.complete().map(CompleteRequest::HeaderProof),
CheckedRequest::HeaderByHash(_, req) => req.complete().map(CompleteRequest::Headers),
CheckedRequest::TransactionIndex(_, req) => req.complete().map(CompleteRequest::TransactionIndex),
CheckedRequest::Receipts(_, req) => req.complete().map(CompleteRequest::Receipts),
CheckedRequest::Body(_, req) => req.complete().map(CompleteRequest::Body),
CheckedRequest::Account(_, req) => req.complete().map(CompleteRequest::Account),
@ -545,6 +559,9 @@ impl net_request::CheckedRequest for CheckedRequest {
CheckedRequest::HeaderByHash(ref prover, _) =>
expect!((&NetResponse::Headers(ref res), &CompleteRequest::Headers(ref req)) =>
prover.check_response(cache, &req.start, &res.headers).map(Response::HeaderByHash)),
CheckedRequest::TransactionIndex(ref prover, _) =>
expect!((&NetResponse::TransactionIndex(ref res), _) =>
prover.check_response(cache, res).map(Response::TransactionIndex)),
CheckedRequest::Receipts(ref prover, _) =>
expect!((&NetResponse::Receipts(ref res), _) =>
prover.check_response(cache, &res.receipts).map(Response::Receipts)),
@ -575,6 +592,8 @@ pub enum Response {
HeaderProof((H256, U256)),
/// Response to a header-by-hash request.
HeaderByHash(encoded::Header),
/// Response to a transaction-index request.
TransactionIndex(net_request::TransactionIndexResponse),
/// Response to a receipts request.
Receipts(Vec<Receipt>),
/// Response to a block body request.
@ -723,6 +742,33 @@ impl HeaderByHash {
}
}
/// Request for a transaction index.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct TransactionIndex(pub Field<H256>);
impl TransactionIndex {
/// Check a response for the transaction index.
//
// TODO: proper checking involves looking at canonicality of the
// hash w.r.t. the current best block header.
//
// unlike all other forms of request, we don't know the header to check
// until we make this request.
//
// This would require lookups in the database or perhaps CHT requests,
// which aren't currently possible.
//
// Also, returning a result that is not locally canonical doesn't necessarily
// indicate misbehavior, so the punishment scheme would need to be revised.
pub fn check_response(
&self,
_cache: &Mutex<::cache::Cache>,
res: &net_request::TransactionIndexResponse,
) -> Result<net_request::TransactionIndexResponse, Error> {
Ok(res.clone())
}
}
/// Request for a block, with header for verification.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct Body(pub HeaderRef);

View File

@ -14,7 +14,7 @@
// You should have received a copy of the GNU General Public License
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
//! A provider for the LES protocol. This is typically a full node, who can
//! A provider for the PIP protocol. This is typically a full node, who can
//! give as much data as necessary to its peers.
use std::sync::Arc;
@ -102,6 +102,10 @@ pub trait Provider: Send + Sync {
/// Get a block header by id.
fn block_header(&self, id: BlockId) -> Option<encoded::Header>;
/// Get a transaction index by hash.
fn transaction_index(&self, req: request::CompleteTransactionIndexRequest)
-> Option<request::TransactionIndexResponse>;
/// Fulfill a block body request.
fn block_body(&self, req: request::CompleteBodyRequest) -> Option<request::BodyResponse>;
@ -150,6 +154,18 @@ impl<T: ProvingBlockChainClient + ?Sized> Provider for T {
BlockChainClient::block_header(self, id)
}
fn transaction_index(&self, req: request::CompleteTransactionIndexRequest)
-> Option<request::TransactionIndexResponse>
{
use ethcore::ids::TransactionId;
self.transaction_receipt(TransactionId::Hash(req.hash)).map(|receipt| request::TransactionIndexResponse {
num: receipt.block_number,
hash: receipt.block_hash,
index: receipt.transaction_index as u64,
})
}
fn block_body(&self, req: request::CompleteBodyRequest) -> Option<request::BodyResponse> {
BlockChainClient::block_body(self, BlockId::Hash(req.hash))
.map(|body| ::request::BodyResponse { body: body })
@ -311,6 +327,12 @@ impl<L: AsLightClient + Send + Sync> Provider for LightProvider<L> {
self.client.as_light_client().block_header(id)
}
fn transaction_index(&self, _req: request::CompleteTransactionIndexRequest)
-> Option<request::TransactionIndexResponse>
{
None
}
fn block_body(&self, _req: request::CompleteBodyRequest) -> Option<request::BodyResponse> {
None
}

View File

@ -23,7 +23,8 @@ use ethcore::encoded;
use ethcore::executed::{Executed, ExecutionError};
use ethcore::ids::BlockId;
use ethcore::filter::Filter as EthcoreFilter;
use ethcore::transaction::{Action, Transaction as EthTransaction};
use ethcore::transaction::{Action, Transaction as EthTransaction, SignedTransaction};
use ethcore::receipt::Receipt;
use jsonrpc_core::{BoxFuture, Error};
use jsonrpc_core::futures::{future, Future};
@ -38,14 +39,18 @@ use light::request::Field;
use ethsync::LightSync;
use bigint::prelude::U256;
use hash::H256;
use util::Address;
use parking_lot::Mutex;
use v1::helpers::{CallRequest as CallRequestHelper, errors, dispatch};
use v1::types::{BlockNumber, CallRequest, Log};
use v1::types::{BlockNumber, CallRequest, Log, Transaction};
const NO_INVALID_BACK_REFS: &'static str = "Fails only on invalid back-references; back-references here known to be valid; qed";
/// Helper for fetching blockchain data either from the light client or the network
/// as necessary.
#[derive(Clone)]
pub struct LightFetch {
/// The light client.
pub client: Arc<LightChainClient>,
@ -57,6 +62,19 @@ pub struct LightFetch {
pub cache: Arc<Mutex<Cache>>,
}
/// Extract a transaction at given index.
pub fn extract_transaction_at_index(block: encoded::Block, index: usize, eip86_transition: u64) -> Option<Transaction> {
block.transactions().into_iter().nth(index)
.and_then(|tx| SignedTransaction::new(tx).ok())
.map(|tx| Transaction::from_signed(tx, block.number(), eip86_transition))
.map(|mut tx| {
tx.block_hash = Some(block.hash().into());
tx.transaction_index = Some(index.into());
tx
})
}
/// Type alias for convenience.
pub type ExecutionResult = Result<Executed, ExecutionError>;
@ -131,7 +149,7 @@ impl LightFetch {
}
}
/// helper for getting account info at a given block.
/// Helper for getting account info at a given block.
/// `None` indicates the account doesn't exist at the given block.
pub fn account(&self, address: Address, id: BlockId) -> BoxFuture<Option<BasicAccount>, Error> {
let mut reqs = Vec::new();
@ -158,7 +176,7 @@ impl LightFetch {
}
}
/// helper for getting proved execution.
/// Helper for getting proved execution.
pub fn proved_execution(&self, req: CallRequest, num: Trailing<BlockNumber>) -> BoxFuture<ExecutionResult, Error> {
const DEFAULT_GAS_PRICE: u64 = 21_000;
// starting gas when gas not provided.
@ -235,7 +253,7 @@ impl LightFetch {
}))
}
/// get a block itself. fails on unknown block ID.
/// Get a block itself. Fails on unknown block ID.
pub fn block(&self, id: BlockId) -> BoxFuture<encoded::Block, Error> {
let mut reqs = Vec::new();
let header_ref = match self.make_header_requests(id, &mut reqs) {
@ -247,7 +265,7 @@ impl LightFetch {
let maybe_future = self.sync.with_context(move |ctx| {
Box::new(self.on_demand.request_raw(ctx, reqs)
.expect("all back-references known to be valid; qed")
.expect(NO_INVALID_BACK_REFS)
.map(|mut res| match res.pop() {
Some(OnDemandResponse::Body(b)) => b,
_ => panic!("responses correspond directly with requests in amount and type; qed"),
@ -261,13 +279,37 @@ impl LightFetch {
}
}
/// get transaction logs
/// Get the block receipts. Fails on unknown block ID.
pub fn receipts(&self, id: BlockId) -> BoxFuture<Vec<Receipt>, Error> {
let mut reqs = Vec::new();
let header_ref = match self.make_header_requests(id, &mut reqs) {
Ok(r) => r,
Err(e) => return Box::new(future::err(e)),
};
reqs.push(request::BlockReceipts(header_ref).into());
let maybe_future = self.sync.with_context(move |ctx| {
Box::new(self.on_demand.request_raw(ctx, reqs)
.expect(NO_INVALID_BACK_REFS)
.map(|mut res| match res.pop() {
Some(OnDemandResponse::Receipts(b)) => b,
_ => panic!("responses correspond directly with requests in amount and type; qed"),
})
.map_err(errors::on_demand_cancel))
});
match maybe_future {
Some(recv) => recv,
None => Box::new(future::err(errors::network_disabled()))
}
}
/// Get transaction logs
pub fn logs(&self, filter: EthcoreFilter) -> BoxFuture<Vec<Log>, Error> {
use std::collections::BTreeMap;
use jsonrpc_core::futures::stream::{self, Stream};
const NO_INVALID_BACK_REFS: &'static str = "Fails only on invalid back-references; back-references here known to be valid; qed";
// early exit for "to" block before "from" block.
let best_number = self.client.chain_info().best_block_number;
let block_number = |id| match id {
@ -318,6 +360,65 @@ impl LightFetch {
None => Box::new(future::err(errors::network_disabled())),
}
}
// Get a transaction by hash. also returns the index in the block.
// Only returns transactions in the canonical chain.
pub fn transaction_by_hash(&self, tx_hash: H256, eip86_transition: u64)
-> BoxFuture<Option<(Transaction, usize)>, Error>
{
let params = (self.sync.clone(), self.on_demand.clone());
let fetcher: Self = self.clone();
Box::new(future::loop_fn(params, move |(sync, on_demand)| {
let maybe_future = sync.with_context(|ctx| {
let req = request::TransactionIndex(tx_hash.clone().into());
on_demand.request(ctx, req)
});
let eventual_index = match maybe_future {
Some(e) => e.expect(NO_INVALID_BACK_REFS).map_err(errors::on_demand_cancel),
None => return Either::A(future::err(errors::network_disabled())),
};
let fetcher = fetcher.clone();
let extract_transaction = eventual_index.and_then(move |index| {
// check that the block is known by number.
// that ensures that it is within the chain that we are aware of.
fetcher.block(BlockId::Number(index.num)).then(move |blk| match blk {
Ok(blk) => {
// if the block is known by number, make sure the
// index from earlier isn't garbage.
if blk.hash() != index.hash {
// index is on a different chain from us.
return Ok(future::Loop::Continue((sync, on_demand)))
}
let index = index.index as usize;
let transaction = extract_transaction_at_index(blk, index, eip86_transition);
if transaction.as_ref().map_or(true, |tx| tx.hash != tx_hash.into()) {
// index is actively wrong: indicated block has
// fewer transactions than necessary or the transaction
// at that index had a different hash.
// TODO: punish peer/move into OnDemand somehow?
Ok(future::Loop::Continue((sync, on_demand)))
} else {
let transaction = transaction.map(move |tx| (tx, index));
Ok(future::Loop::Break(transaction))
}
}
Err(ref e) if e == &errors::unknown_block() => {
// block by number not in the canonical chain.
Ok(future::Loop::Break(None))
}
Err(e) => Err(e),
})
});
Either::B(extract_transaction)
}))
}
}
#[derive(Clone)]

View File

@ -458,38 +458,54 @@ impl<C, SN: ?Sized, S: ?Sized, M, EM> Eth for EthClient<C, SN, S, M, EM> where
Box::new(future::done(self.block(num.into(), include_txs)))
}
fn transaction_by_hash(&self, hash: RpcH256) -> Result<Option<Transaction>, Error> {
fn transaction_by_hash(&self, hash: RpcH256) -> BoxFuture<Option<Transaction>, Error> {
let hash: H256 = hash.into();
let block_number = self.client.chain_info().best_block_number;
Ok(self.transaction(TransactionId::Hash(hash))?.or_else(|| self.miner.transaction(block_number, &hash).map(|t| Transaction::from_pending(t, block_number, self.eip86_transition))))
let tx = try_bf!(self.transaction(TransactionId::Hash(hash))).or_else(|| {
self.miner.transaction(block_number, &hash)
.map(|t| Transaction::from_pending(t, block_number, self.eip86_transition))
});
Box::new(future::ok(tx))
}
fn transaction_by_block_hash_and_index(&self, hash: RpcH256, index: Index) -> Result<Option<Transaction>, Error> {
self.transaction(TransactionId::Location(BlockId::Hash(hash.into()), index.value()))
fn transaction_by_block_hash_and_index(&self, hash: RpcH256, index: Index) -> BoxFuture<Option<Transaction>, Error> {
Box::new(future::done(
self.transaction(TransactionId::Location(BlockId::Hash(hash.into()), index.value()))
))
}
fn transaction_by_block_number_and_index(&self, num: BlockNumber, index: Index) -> Result<Option<Transaction>, Error> {
self.transaction(TransactionId::Location(num.into(), index.value()))
fn transaction_by_block_number_and_index(&self, num: BlockNumber, index: Index) -> BoxFuture<Option<Transaction>, Error> {
Box::new(future::done(
self.transaction(TransactionId::Location(num.into(), index.value()))
))
}
fn transaction_receipt(&self, hash: RpcH256) -> Result<Option<Receipt>, Error> {
fn transaction_receipt(&self, hash: RpcH256) -> BoxFuture<Option<Receipt>, Error> {
let best_block = self.client.chain_info().best_block_number;
let hash: H256 = hash.into();
match (self.miner.pending_receipt(best_block, &hash), self.options.allow_pending_receipt_query) {
(Some(receipt), true) => Ok(Some(receipt.into())),
(Some(receipt), true) => Box::new(future::ok(Some(receipt.into()))),
_ => {
let receipt = self.client.transaction_receipt(TransactionId::Hash(hash));
Ok(receipt.map(Into::into))
Box::new(future::ok(receipt.map(Into::into)))
}
}
}
fn uncle_by_block_hash_and_index(&self, hash: RpcH256, index: Index) -> Result<Option<RichBlock>, Error> {
self.uncle(UncleId { block: BlockId::Hash(hash.into()), position: index.value() })
fn uncle_by_block_hash_and_index(&self, hash: RpcH256, index: Index) -> BoxFuture<Option<RichBlock>, Error> {
Box::new(future::done(self.uncle(UncleId {
block: BlockId::Hash(hash.into()),
position: index.value()
})))
}
fn uncle_by_block_number_and_index(&self, num: BlockNumber, index: Index) -> Result<Option<RichBlock>, Error> {
self.uncle(UncleId { block: num.into(), position: index.value() })
fn uncle_by_block_number_and_index(&self, num: BlockNumber, index: Index) -> BoxFuture<Option<RichBlock>, Error> {
Box::new(future::done(self.uncle(UncleId {
block: num.into(),
position: index.value()
})))
}
fn compilers(&self) -> Result<Vec<String>, Error> {

View File

@ -39,11 +39,10 @@ use hash::{KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP};
use bigint::prelude::U256;
use parking_lot::{RwLock, Mutex};
use v1::impls::eth_filter::Filterable;
use v1::helpers::{errors, limit_logs};
use v1::helpers::{PollFilter, PollManager};
use v1::helpers::light_fetch::LightFetch;
use v1::helpers::light_fetch::{self, LightFetch};
use v1::traits::Eth;
use v1::types::{
RichBlock, Block, BlockTransactions, BlockNumber, Bytes, SyncStatus, SyncInfo,
@ -80,7 +79,6 @@ impl<T> Clone for EthClient<T> {
}
}
impl<T: LightChainClient + 'static> EthClient<T> {
/// 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.
@ -393,33 +391,72 @@ impl<T: LightChainClient + 'static> Eth for EthClient<T> {
}))
}
fn transaction_by_hash(&self, _hash: RpcH256) -> Result<Option<Transaction>, Error> {
Err(errors::unimplemented(None))
fn transaction_by_hash(&self, hash: RpcH256) -> BoxFuture<Option<Transaction>, Error> {
let eip86 = self.client.eip86_transition();
Box::new(self.fetcher().transaction_by_hash(hash.into(), eip86).map(|x| x.map(|(tx, _)| tx)))
}
fn transaction_by_block_hash_and_index(&self, _hash: RpcH256, _idx: Index) -> Result<Option<Transaction>, Error> {
Err(errors::unimplemented(None))
fn transaction_by_block_hash_and_index(&self, hash: RpcH256, idx: Index) -> BoxFuture<Option<Transaction>, Error> {
let eip86 = self.client.eip86_transition();
Box::new(self.fetcher().block(BlockId::Hash(hash.into())).map(move |block| {
light_fetch::extract_transaction_at_index(block, idx.value(), eip86)
}))
}
fn transaction_by_block_number_and_index(&self, _num: BlockNumber, _idx: Index) -> Result<Option<Transaction>, Error> {
Err(errors::unimplemented(None))
fn transaction_by_block_number_and_index(&self, num: BlockNumber, idx: Index) -> BoxFuture<Option<Transaction>, Error> {
let eip86 = self.client.eip86_transition();
Box::new(self.fetcher().block(num.into()).map(move |block| {
light_fetch::extract_transaction_at_index(block, idx.value(), eip86)
}))
}
fn transaction_receipt(&self, _hash: RpcH256) -> Result<Option<Receipt>, Error> {
Err(errors::unimplemented(None))
fn transaction_receipt(&self, hash: RpcH256) -> BoxFuture<Option<Receipt>, Error> {
let eip86 = self.client.eip86_transition();
let fetcher = self.fetcher();
Box::new(fetcher.transaction_by_hash(hash.clone().into(), eip86).and_then(move |tx| {
// the block hash included in the transaction object here has
// already been checked for canonicality and whether it contains
// the transaction.
match tx {
Some((tx, index)) => match tx.block_hash.clone() {
Some(block_hash) => {
let extract_receipt = fetcher.receipts(BlockId::Hash(block_hash.clone().into()))
.and_then(move |mut receipts| future::ok(receipts.swap_remove(index)))
.map(Receipt::from)
.map(move |mut receipt| {
receipt.transaction_hash = Some(hash);
receipt.transaction_index = Some(index.into());
receipt.block_hash = Some(block_hash);
receipt.block_number = tx.block_number;
receipt
})
.map(Some);
Either::B(extract_receipt)
}
None => Either::A(future::err(errors::unknown_block())),
},
None => Either::A(future::ok(None)),
}
}))
}
fn uncle_by_block_hash_and_index(&self, _hash: RpcH256, _idx: Index) -> Result<Option<RichBlock>, Error> {
Err(errors::unimplemented(None))
fn uncle_by_block_hash_and_index(&self, hash: RpcH256, idx: Index) -> BoxFuture<Option<RichBlock>, Error> {
let client = self.client.clone();
Box::new(self.fetcher().block(BlockId::Hash(hash.into())).map(move |block| {
extract_uncle_at_index(block, idx, client)
}))
}
fn uncle_by_block_number_and_index(&self, _num: BlockNumber, _idx: Index) -> Result<Option<RichBlock>, Error> {
Err(errors::unimplemented(None))
fn uncle_by_block_number_and_index(&self, num: BlockNumber, idx: Index) -> BoxFuture<Option<RichBlock>, Error> {
let client = self.client.clone();
Box::new(self.fetcher().block(num.into()).map(move |block| {
extract_uncle_at_index(block, idx, client)
}))
}
fn compilers(&self) -> Result<Vec<String>, Error> {
Err(errors::deprecated("Compilation functionality is deprecated.".to_string()))
}
fn compile_lll(&self, _: String) -> Result<Bytes, Error> {
@ -478,3 +515,37 @@ impl<T: LightChainClient + 'static> Filterable for EthClient<T> {
&self.polls
}
}
fn extract_uncle_at_index<T: LightChainClient>(block: encoded::Block, index: Index, client: Arc<T>) -> Option<RichBlock> {
let uncle = match block.uncles().into_iter().nth(index.value()) {
Some(u) => u,
None => return None,
};
let extra_info = client.engine().extra_info(&uncle);
Some(RichBlock {
inner: Block {
hash: Some(uncle.hash().into()),
size: None,
parent_hash: uncle.parent_hash().clone().into(),
uncles_hash: uncle.uncles_hash().clone().into(),
author: uncle.author().clone().into(),
miner: uncle.author().clone().into(),
state_root: uncle.state_root().clone().into(),
transactions_root: uncle.transactions_root().clone().into(),
number: Some(uncle.number().into()),
gas_used: uncle.gas_used().clone().into(),
gas_limit: uncle.gas_limit().clone().into(),
logs_bloom: uncle.log_bloom().clone().into(),
timestamp: uncle.timestamp().into(),
difficulty: uncle.difficulty().clone().into(),
total_difficulty: None,
receipts_root: uncle.receipts_root().clone().into(),
extra_data: uncle.extra_data().clone().into(),
seal_fields: uncle.seal().into_iter().cloned().map(Into::into).collect(),
uncles: vec![],
transactions: BlockTransactions::Hashes(vec![]),
},
extra_info: extra_info,
})
}

View File

@ -547,7 +547,7 @@ fn rpc_eth_pending_transaction_by_hash() {
tester.miner.pending_transactions.lock().insert(H256::zero(), tx);
}
let response = r#"{"jsonrpc":"2.0","result":{"blockHash":null,"blockNumber":null,"chainId":null,"condition":null,"creates":null,"from":"0x0f65fe9276bc9a24ae7083ae28e2660ef72df99e","gas":"0x5208","gasPrice":"0x1","hash":"0x41df922fd0d4766fcc02e161f8295ec28522f329ae487f14d811e4b64c8d6e31","input":"0x","nonce":"0x0","publicKey":"0x7ae46da747962c2ee46825839c1ef9298e3bd2e70ca2938495c3693a485ec3eaa8f196327881090ff64cf4fbb0a48485d4f83098e189ed3b7a87d5941b59f789","r":"0x48b55bfa915ac795c431978d8a6a992b628d557da5ff759b307d495a36649353","raw":"0xf85f800182520894095e7baea6a6c7c4c2dfeb977efac326af552d870a801ba048b55bfa915ac795c431978d8a6a992b628d557da5ff759b307d495a36649353a0efffd310ac743f371de3b9f7f9cb56c0b28ad43601b4ab949f53faa07bd2c804","s":"0xefffd310ac743f371de3b9f7f9cb56c0b28ad43601b4ab949f53faa07bd2c804","standardV":"0x0","to":"0x095e7baea6a6c7c4c2dfeb977efac326af552d87","transactionIndex":null,"v":"0x1b","value":"0xa"},"id":1}"#;
let response = r#"{"jsonrpc":"2.0","result":{"blockHash":null,"blockNumber":"0x0","chainId":null,"condition":null,"creates":null,"from":"0x0f65fe9276bc9a24ae7083ae28e2660ef72df99e","gas":"0x5208","gasPrice":"0x1","hash":"0x41df922fd0d4766fcc02e161f8295ec28522f329ae487f14d811e4b64c8d6e31","input":"0x","nonce":"0x0","publicKey":"0x7ae46da747962c2ee46825839c1ef9298e3bd2e70ca2938495c3693a485ec3eaa8f196327881090ff64cf4fbb0a48485d4f83098e189ed3b7a87d5941b59f789","r":"0x48b55bfa915ac795c431978d8a6a992b628d557da5ff759b307d495a36649353","raw":"0xf85f800182520894095e7baea6a6c7c4c2dfeb977efac326af552d870a801ba048b55bfa915ac795c431978d8a6a992b628d557da5ff759b307d495a36649353a0efffd310ac743f371de3b9f7f9cb56c0b28ad43601b4ab949f53faa07bd2c804","s":"0xefffd310ac743f371de3b9f7f9cb56c0b28ad43601b4ab949f53faa07bd2c804","standardV":"0x0","to":"0x095e7baea6a6c7c4c2dfeb977efac326af552d87","transactionIndex":null,"v":"0x1b","value":"0xa"},"id":1}"#;
let request = r#"{
"jsonrpc": "2.0",
"method": "eth_getTransactionByHash",
@ -863,7 +863,7 @@ fn rpc_eth_sign_transaction() {
let response = r#"{"jsonrpc":"2.0","result":{"#.to_owned() +
r#""raw":"0x"# + &rlp.to_hex() + r#"","# +
r#""tx":{"# +
r#""blockHash":null,"blockNumber":null,"# +
r#""blockHash":null,"blockNumber":"0x0","# +
&format!("\"chainId\":{},", t.chain_id().map_or("null".to_owned(), |n| format!("{}", n))) +
r#""condition":null,"creates":null,"# +
&format!("\"from\":\"0x{:?}\",", &address) +

View File

@ -234,7 +234,7 @@ fn rpc_parity_remove_transaction() {
let hash = signed.hash();
let request = r#"{"jsonrpc": "2.0", "method": "parity_removeTransaction", "params":[""#.to_owned() + &format!("0x{:?}", hash) + r#""], "id": 1}"#;
let response = r#"{"jsonrpc":"2.0","result":{"blockHash":null,"blockNumber":null,"chainId":null,"condition":null,"creates":null,"from":"0x0000000000000000000000000000000000000002","gas":"0x76c0","gasPrice":"0x9184e72a000","hash":"0xa2e0da8a8064e0b9f93e95a53c2db6d01280efb8ac72a708d25487e67dd0f8fc","input":"0x","nonce":"0x1","publicKey":null,"r":"0x1","raw":"0xe9018609184e72a0008276c0940000000000000000000000000000000000000005849184e72a80800101","s":"0x1","standardV":"0x4","to":"0x0000000000000000000000000000000000000005","transactionIndex":null,"v":"0x0","value":"0x9184e72a"},"id":1}"#;
let response = r#"{"jsonrpc":"2.0","result":{"blockHash":null,"blockNumber":"0x0","chainId":null,"condition":null,"creates":null,"from":"0x0000000000000000000000000000000000000002","gas":"0x76c0","gasPrice":"0x9184e72a000","hash":"0xa2e0da8a8064e0b9f93e95a53c2db6d01280efb8ac72a708d25487e67dd0f8fc","input":"0x","nonce":"0x1","publicKey":null,"r":"0x1","raw":"0xe9018609184e72a0008276c0940000000000000000000000000000000000000005849184e72a80800101","s":"0x1","standardV":"0x4","to":"0x0000000000000000000000000000000000000005","transactionIndex":null,"v":"0x0","value":"0x9184e72a"},"id":1}"#;
miner.pending_transactions.lock().insert(hash, signed);
assert_eq!(io.handle_request_sync(&request), Some(response.to_owned()));

View File

@ -456,7 +456,7 @@ fn should_confirm_sign_transaction_with_rlp() {
let response = r#"{"jsonrpc":"2.0","result":{"#.to_owned() +
r#""raw":"0x"# + &rlp.to_hex() + r#"","# +
r#""tx":{"# +
r#""blockHash":null,"blockNumber":null,"# +
r#""blockHash":null,"blockNumber":"0x0","# +
&format!("\"chainId\":{},", t.chain_id().map_or("null".to_owned(), |n| format!("{}", n))) +
r#""condition":null,"creates":null,"# +
&format!("\"from\":\"0x{:?}\",", &address) +

View File

@ -299,7 +299,7 @@ fn should_add_sign_transaction_to_the_queue() {
let response = r#"{"jsonrpc":"2.0","result":{"#.to_owned() +
r#""raw":"0x"# + &rlp.to_hex() + r#"","# +
r#""tx":{"# +
r#""blockHash":null,"blockNumber":null,"# +
r#""blockHash":null,"blockNumber":"0x0","# +
&format!("\"chainId\":{},", t.chain_id().map_or("null".to_owned(), |n| format!("{}", n))) +
r#""condition":null,"creates":null,"# +
&format!("\"from\":\"0x{:?}\",", &address) +

View File

@ -117,27 +117,27 @@ build_rpc_trait! {
/// Get transaction by its hash.
#[rpc(name = "eth_getTransactionByHash")]
fn transaction_by_hash(&self, H256) -> Result<Option<Transaction>, Error>;
fn transaction_by_hash(&self, H256) -> BoxFuture<Option<Transaction>, Error>;
/// Returns transaction at given block hash and index.
#[rpc(name = "eth_getTransactionByBlockHashAndIndex")]
fn transaction_by_block_hash_and_index(&self, H256, Index) -> Result<Option<Transaction>, Error>;
fn transaction_by_block_hash_and_index(&self, H256, Index) -> BoxFuture<Option<Transaction>, Error>;
/// Returns transaction by given block number and index.
#[rpc(name = "eth_getTransactionByBlockNumberAndIndex")]
fn transaction_by_block_number_and_index(&self, BlockNumber, Index) -> Result<Option<Transaction>, Error>;
fn transaction_by_block_number_and_index(&self, BlockNumber, Index) -> BoxFuture<Option<Transaction>, Error>;
/// Returns transaction receipt.
/// Returns transaction receipt by transaction hash.
#[rpc(name = "eth_getTransactionReceipt")]
fn transaction_receipt(&self, H256) -> Result<Option<Receipt>, Error>;
fn transaction_receipt(&self, H256) -> BoxFuture<Option<Receipt>, Error>;
/// Returns an uncles at given block and index.
#[rpc(name = "eth_getUncleByBlockHashAndIndex")]
fn uncle_by_block_hash_and_index(&self, H256, Index) -> Result<Option<RichBlock>, Error>;
fn uncle_by_block_hash_and_index(&self, H256, Index) -> BoxFuture<Option<RichBlock>, Error>;
/// Returns an uncles at given block and index.
#[rpc(name = "eth_getUncleByBlockNumberAndIndex")]
fn uncle_by_block_number_and_index(&self, BlockNumber, Index) -> Result<Option<RichBlock>, Error>;
fn uncle_by_block_number_and_index(&self, BlockNumber, Index) -> BoxFuture<Option<RichBlock>, Error>;
/// Returns available compilers.
/// @deprecated

View File

@ -213,7 +213,7 @@ impl Transaction {
hash: t.hash().into(),
nonce: t.nonce.into(),
block_hash: None,
block_number: None,
block_number: Some(block_number.into()),
transaction_index: None,
from: t.sender().into(),
to: match t.action {