Merge pull request #3897 from ethcore/single-provider-functions

require only simpler methods on Provider
This commit is contained in:
Gav Wood 2016-12-19 22:23:39 +01:00 committed by GitHub
commit 5ba16e4867
4 changed files with 206 additions and 193 deletions

View File

@ -58,7 +58,7 @@ impl Client {
/// Import a local transaction. /// Import a local transaction.
pub fn import_own_transaction(&self, tx: SignedTransaction) { pub fn import_own_transaction(&self, tx: SignedTransaction) {
self.tx_pool.lock().insert(tx.hash(), tx); self.tx_pool.lock().insert(tx.hash(), tx);
} }
/// Fetch a vector of all pending transactions. /// Fetch a vector of all pending transactions.
pub fn pending_transactions(&self) -> Vec<SignedTransaction> { pub fn pending_transactions(&self) -> Vec<SignedTransaction> {
@ -90,28 +90,28 @@ impl Provider for Client {
None None
} }
fn block_headers(&self, _req: request::Headers) -> Vec<Bytes> { fn block_header(&self, _id: BlockId) -> Option<Bytes> {
None
}
fn block_body(&self, _id: BlockId) -> Option<Bytes> {
None
}
fn block_receipts(&self, _hash: &H256) -> Option<Bytes> {
None
}
fn state_proof(&self, _req: request::StateProof) -> Vec<Bytes> {
Vec::new() Vec::new()
} }
fn block_bodies(&self, _req: request::Bodies) -> Vec<Bytes> { fn contract_code(&self, _req: request::ContractCode) -> Bytes {
Vec::new() Vec::new()
} }
fn receipts(&self, _req: request::Receipts) -> Vec<Bytes> { fn header_proof(&self, _req: request::HeaderProof) -> Option<(Bytes, Vec<Bytes>)> {
Vec::new() None
}
fn proofs(&self, _req: request::StateProofs) -> Vec<Bytes> {
Vec::new()
}
fn contract_code(&self, _req: request::ContractCodes) -> Vec<Bytes> {
Vec::new()
}
fn header_proofs(&self, _req: request::HeaderProofs) -> Vec<Bytes> {
Vec::new()
} }
fn ready_transactions(&self) -> Vec<PendingTransaction> { fn ready_transactions(&self) -> Vec<PendingTransaction> {

View File

@ -157,7 +157,7 @@ impl Peer {
/// An LES event handler. /// An LES event handler.
/// ///
/// Each handler function takes a context which describes the relevant peer /// Each handler function takes a context which describes the relevant peer
/// and gives references to the IO layer and protocol structure so new messages /// and gives references to the IO layer and protocol structure so new messages
/// can be dispatched immediately. /// can be dispatched immediately.
/// ///
@ -185,7 +185,7 @@ pub trait Handler: Send + Sync {
fn on_state_proofs(&self, _ctx: &EventContext, _req_id: ReqId, _proofs: &[Vec<Bytes>]) { } fn on_state_proofs(&self, _ctx: &EventContext, _req_id: ReqId, _proofs: &[Vec<Bytes>]) { }
/// Called when a peer responds with contract code. /// Called when a peer responds with contract code.
fn on_code(&self, _ctx: &EventContext, _req_id: ReqId, _codes: &[Bytes]) { } fn on_code(&self, _ctx: &EventContext, _req_id: ReqId, _codes: &[Bytes]) { }
/// Called when a peer responds with header proofs. Each proof is a block header coupled /// Called when a peer responds with header proofs. Each proof is a block header coupled
/// with a series of trie nodes is ascending order by distance from the root. /// with a series of trie nodes is ascending order by distance from the root.
fn on_header_proofs(&self, _ctx: &EventContext, _req_id: ReqId, _proofs: &[(Bytes, Vec<Bytes>)]) { } fn on_header_proofs(&self, _ctx: &EventContext, _req_id: ReqId, _proofs: &[(Bytes, Vec<Bytes>)]) { }
/// Called on abort. /// Called on abort.
@ -215,9 +215,9 @@ pub struct Params {
/// This is simply designed for request-response purposes. Higher level uses /// This is simply designed for request-response purposes. Higher level uses
/// of the protocol, such as synchronization, will function as wrappers around /// of the protocol, such as synchronization, will function as wrappers around
/// this system. /// this system.
// //
// LOCK ORDER: // LOCK ORDER:
// Locks must be acquired in the order declared, and when holding a read lock // Locks must be acquired in the order declared, and when holding a read lock
// on the peers, only one peer may be held at a time. // on the peers, only one peer may be held at a time.
pub struct LightProtocol { pub struct LightProtocol {
provider: Arc<Provider>, provider: Arc<Provider>,
@ -252,7 +252,7 @@ impl LightProtocol {
} }
} }
/// Check the maximum amount of requests of a specific type /// Check the maximum amount of requests of a specific type
/// which a peer would be able to serve. /// which a peer would be able to serve.
pub fn max_requests(&self, peer: PeerId, kind: request::Kind) -> Option<usize> { pub fn max_requests(&self, peer: PeerId, kind: request::Kind) -> Option<usize> {
self.peers.read().get(&peer).and_then(|peer| { self.peers.read().get(&peer).and_then(|peer| {
@ -267,11 +267,11 @@ impl LightProtocol {
}) })
} }
/// Make a request to a peer. /// Make a request to a peer.
/// ///
/// Fails on: nonexistent peer, network error, peer not server, /// Fails on: nonexistent peer, network error, peer not server,
/// insufficient buffer. Does not check capabilities before sending. /// insufficient buffer. Does not check capabilities before sending.
/// On success, returns a request id which can later be coordinated /// On success, returns a request id which can later be coordinated
/// with an event. /// with an event.
pub fn request_from(&self, io: &IoContext, peer_id: &PeerId, request: Request) -> Result<ReqId, Error> { pub fn request_from(&self, io: &IoContext, peer_id: &PeerId, request: Request) -> Result<ReqId, Error> {
let peers = self.peers.read(); let peers = self.peers.read();
@ -325,10 +325,10 @@ impl LightProtocol {
// TODO: "urgent" announcements like new blocks? // TODO: "urgent" announcements like new blocks?
// the timer approach will skip 1 (possibly 2) in rare occasions. // the timer approach will skip 1 (possibly 2) in rare occasions.
if peer_info.sent_head == announcement.head_hash || if peer_info.sent_head == announcement.head_hash ||
peer_info.status.head_num >= announcement.head_num || peer_info.status.head_num >= announcement.head_num ||
now - peer_info.last_update < Duration::milliseconds(UPDATE_INTERVAL_MS) { now - peer_info.last_update < Duration::milliseconds(UPDATE_INTERVAL_MS) {
continue continue
} }
peer_info.last_update = now; peer_info.last_update = now;
@ -357,7 +357,7 @@ impl LightProtocol {
/// Add an event handler. /// Add an event handler.
/// Ownership will be transferred to the protocol structure, /// Ownership will be transferred to the protocol structure,
/// and the handler will be kept alive as long as it is. /// and the handler will be kept alive as long as it is.
/// These are intended to be added when the protocol structure /// These are intended to be added when the protocol structure
/// is initialized as a means of customizing its behavior. /// is initialized as a means of customizing its behavior.
pub fn add_handler(&mut self, handler: Box<Handler>) { pub fn add_handler(&mut self, handler: Box<Handler>) {
self.handlers.push(handler); self.handlers.push(handler);
@ -380,7 +380,7 @@ impl LightProtocol {
pending_requests.clear(); pending_requests.clear();
} }
// Does the common pre-verification of responses before the response itself // Does the common pre-verification of responses before the response itself
// is actually decoded: // is actually decoded:
// - check whether peer exists // - check whether peer exists
// - check whether request was made // - check whether request was made
@ -406,7 +406,7 @@ impl LightProtocol {
let mut peer_info = peer_info.lock(); let mut peer_info = peer_info.lock();
match peer_info.remote_flow.as_mut() { match peer_info.remote_flow.as_mut() {
Some(&mut (ref mut buf, ref mut flow)) => { Some(&mut (ref mut buf, ref mut flow)) => {
let actual_buffer = ::std::cmp::min(cur_buffer, *flow.limit()); let actual_buffer = ::std::cmp::min(cur_buffer, *flow.limit());
buf.update_to(actual_buffer) buf.update_to(actual_buffer)
} }
None => return Err(Error::NotServer), // this really should be impossible. None => return Err(Error::NotServer), // this really should be impossible.
@ -488,17 +488,17 @@ impl LightProtocol {
request::Kind::Receipts => timeout::RECEIPTS, request::Kind::Receipts => timeout::RECEIPTS,
request::Kind::StateProofs => timeout::PROOFS, request::Kind::StateProofs => timeout::PROOFS,
request::Kind::Codes => timeout::CONTRACT_CODES, request::Kind::Codes => timeout::CONTRACT_CODES,
request::Kind::HeaderProofs => timeout::HEADER_PROOFS, request::Kind::HeaderProofs => timeout::HEADER_PROOFS,
}; };
if r.timestamp + Duration::milliseconds(kind_timeout) <= now { if r.timestamp + Duration::milliseconds(kind_timeout) <= now {
debug!(target: "les", "Request for {:?} from peer {} timed out", debug!(target: "les", "Request for {:?} from peer {} timed out",
r.request.kind(), r.peer_id); r.request.kind(), r.peer_id);
// keep the request in the `pending` set for now so // keep the request in the `pending` set for now so
// on_disconnect will pass unfulfilled ReqIds to handlers. // on_disconnect will pass unfulfilled ReqIds to handlers.
// in the case that a response is received after this, the // in the case that a response is received after this, the
// disconnect won't be cancelled but the ReqId won't be // disconnect won't be cancelled but the ReqId won't be
// marked as abandoned. // marked as abandoned.
io.disconnect_peer(r.peer_id); io.disconnect_peer(r.peer_id);
} }
@ -519,7 +519,7 @@ impl LightProtocol {
punish(*peer, io, Error::UnsupportedProtocolVersion(proto_version)); punish(*peer, io, Error::UnsupportedProtocolVersion(proto_version));
return; return;
} }
let chain_info = self.provider.chain_info(); let chain_info = self.provider.chain_info();
let status = Status { let status = Status {
@ -540,7 +540,7 @@ impl LightProtocol {
last_update: SteadyTime::now(), last_update: SteadyTime::now(),
}); });
io.send(*peer, packet::STATUS, status_packet); io.send(*peer, packet::STATUS, status_packet);
} }
// called when a peer disconnects. // called when a peer disconnects.
@ -569,7 +569,7 @@ impl LightProtocol {
io: io, io: io,
proto: self, proto: self,
}, &unfulfilled) }, &unfulfilled)
} }
} }
} }
@ -608,7 +608,7 @@ impl LightProtocol {
for handler in &self.handlers { for handler in &self.handlers {
handler.on_connect(&Ctx { handler.on_connect(&Ctx {
peer: *peer, peer: *peer,
io: io, io: io,
proto: self, proto: self,
}, &status, &capabilities) }, &status, &capabilities)
} }
@ -662,7 +662,7 @@ impl LightProtocol {
} }
// Handle a request for block headers. // Handle a request for block headers.
fn get_block_headers(&self, peer: &PeerId, io: &IoContext, data: UntrustedRlp) -> Result<(), Error> { fn get_block_headers(&self, peer: &PeerId, io: &IoContext, data: UntrustedRlp) -> Result<(), Error> {
const MAX_HEADERS: usize = 512; const MAX_HEADERS: usize = 512;
let peers = self.peers.read(); let peers = self.peers.read();
@ -914,7 +914,7 @@ impl LightProtocol {
.map(|x| x.iter().map(|node| node.as_raw().to_owned()).collect()) .map(|x| x.iter().map(|node| node.as_raw().to_owned()).collect())
.collect(); .collect();
for handler in &self.handlers { for handler in &self.handlers {
handler.on_state_proofs(&Ctx { handler.on_state_proofs(&Ctx {
peer: *peer, peer: *peer,
io: io, io: io,
@ -956,7 +956,7 @@ impl LightProtocol {
let max_cost = try!(peer.deduct_max(&self.flow_params, request::Kind::Codes, req.code_requests.len())); let max_cost = try!(peer.deduct_max(&self.flow_params, request::Kind::Codes, req.code_requests.len()));
let response = self.provider.contract_code(req); let response = self.provider.contract_codes(req);
let response_len = response.iter().filter(|x| !x.is_empty()).count(); let response_len = response.iter().filter(|x| !x.is_empty()).count();
let actual_cost = self.flow_params.compute_cost(request::Kind::Codes, response_len); let actual_cost = self.flow_params.compute_cost(request::Kind::Codes, response_len);
assert!(max_cost >= actual_cost, "Actual cost exceeded maximum computed cost."); assert!(max_cost >= actual_cost, "Actual cost exceeded maximum computed cost.");
@ -983,7 +983,7 @@ impl LightProtocol {
let raw_code: Vec<Bytes> = try!(try!(raw.at(2)).iter().map(|x| x.as_val()).collect()); let raw_code: Vec<Bytes> = try!(try!(raw.at(2)).iter().map(|x| x.as_val()).collect());
for handler in &self.handlers { for handler in &self.handlers {
handler.on_code(&Ctx { handler.on_code(&Ctx {
peer: *peer, peer: *peer,
io: io, io: io,
@ -1055,11 +1055,11 @@ impl LightProtocol {
try!(raw.at(1)).iter().map(|x| x.as_raw().to_owned()).collect(), try!(raw.at(1)).iter().map(|x| x.as_raw().to_owned()).collect(),
)) ))
} }
let req_id = try!(self.pre_verify_response(peer, request::Kind::HeaderProofs, &raw)); let req_id = try!(self.pre_verify_response(peer, request::Kind::HeaderProofs, &raw));
let raw_proofs: Vec<_> = try!(try!(raw.at(2)).iter().map(decode_res).collect()); let raw_proofs: Vec<_> = try!(try!(raw.at(2)).iter().map(decode_res).collect());
for handler in &self.handlers { for handler in &self.handlers {
handler.on_header_proofs(&Ctx { handler.on_header_proofs(&Ctx {
peer: *peer, peer: *peer,
io: io, io: io,
@ -1082,7 +1082,7 @@ impl LightProtocol {
handler.on_transactions(&Ctx { handler.on_transactions(&Ctx {
peer: *peer, peer: *peer,
io: io, io: io,
proto: self, proto: self,
}, &txs); }, &txs);
} }
@ -1136,12 +1136,12 @@ fn encode_request(req: &Request, req_id: usize) -> Vec<u8> {
Request::Headers(ref headers) => { Request::Headers(ref headers) => {
let mut stream = RlpStream::new_list(2); let mut stream = RlpStream::new_list(2);
stream.append(&req_id).begin_list(4); stream.append(&req_id).begin_list(4);
match headers.start { match headers.start {
HashOrNumber::Hash(ref hash) => stream.append(hash), HashOrNumber::Hash(ref hash) => stream.append(hash),
HashOrNumber::Number(ref num) => stream.append(num), HashOrNumber::Number(ref num) => stream.append(num),
}; };
stream stream
.append(&headers.max) .append(&headers.max)
.append(&headers.skip) .append(&headers.skip)
@ -1214,4 +1214,4 @@ fn encode_request(req: &Request, req_id: usize) -> Vec<u8> {
stream.out() stream.out()
} }
} }
} }

View File

@ -94,79 +94,36 @@ impl Provider for TestProvider {
None None
} }
fn block_headers(&self, req: request::Headers) -> Vec<Bytes> { fn block_header(&self, id: BlockId) -> Option<Bytes> {
use request::HashOrNumber; self.0.client.block_header(id)
use ethcore::views::HeaderView; }
let best_num = self.chain_info().best_block_number; fn block_body(&self, id: BlockId) -> Option<Bytes> {
let start_num = match req.start { self.0.client.block_body(id)
HashOrNumber::Number(start_num) => start_num, }
HashOrNumber::Hash(hash) => match self.0.client.block_header(BlockId::Hash(hash)) {
None => {
return Vec::new();
}
Some(header) => {
let num = HeaderView::new(&header).number();
if req.max == 1 || self.0.client.block_hash(BlockId::Number(num)) != Some(hash) {
// Non-canonical header or single header requested.
return vec![header];
}
num fn block_receipts(&self, hash: &H256) -> Option<Bytes> {
} self.0.client.block_receipts(&hash)
}
fn state_proof(&self, req: request::StateProof) -> Vec<Bytes> {
match req.key2 {
Some(_) => vec![::util::sha3::SHA3_NULL_RLP.to_vec()],
None => {
// sort of a leaf node
let mut stream = RlpStream::new_list(2);
stream.append(&req.key1).append_empty_data();
vec![stream.out()]
} }
}; }
(0u64..req.max as u64)
.map(|x: u64| x.saturating_mul(req.skip + 1))
.take_while(|x| if req.reverse { x < &start_num } else { best_num - start_num >= *x })
.map(|x| if req.reverse { start_num - x } else { start_num + x })
.map(|x| self.0.client.block_header(BlockId::Number(x)))
.take_while(|x| x.is_some())
.flat_map(|x| x)
.collect()
} }
fn block_bodies(&self, req: request::Bodies) -> Vec<Bytes> { fn contract_code(&self, req: request::ContractCode) -> Bytes {
req.block_hashes.into_iter() req.account_key.iter().chain(req.account_key.iter()).cloned().collect()
.map(|hash| self.0.client.block_body(BlockId::Hash(hash)))
.map(|body| body.unwrap_or_else(|| ::rlp::EMPTY_LIST_RLP.to_vec()))
.collect()
} }
fn receipts(&self, req: request::Receipts) -> Vec<Bytes> { fn header_proof(&self, _req: request::HeaderProof) -> Option<(Bytes, Vec<Bytes>)> {
req.block_hashes.into_iter() None
.map(|hash| self.0.client.block_receipts(&hash))
.map(|receipts| receipts.unwrap_or_else(|| ::rlp::EMPTY_LIST_RLP.to_vec()))
.collect()
}
fn proofs(&self, req: request::StateProofs) -> Vec<Bytes> {
req.requests.into_iter()
.map(|req| {
match req.key2 {
Some(_) => ::util::sha3::SHA3_NULL_RLP.to_vec(),
None => {
// sort of a leaf node
let mut stream = RlpStream::new_list(2);
stream.append(&req.key1).append_empty_data();
stream.out()
}
}
})
.collect()
}
fn contract_code(&self, req: request::ContractCodes) -> Vec<Bytes> {
req.code_requests.into_iter()
.map(|req| {
req.account_key.iter().chain(req.account_key.iter()).cloned().collect()
})
.collect()
}
fn header_proofs(&self, req: request::HeaderProofs) -> Vec<Bytes> {
req.requests.into_iter().map(|_| ::rlp::EMPTY_LIST_RLP.to_vec()).collect()
} }
fn ready_transactions(&self) -> Vec<PendingTransaction> { fn ready_transactions(&self) -> Vec<PendingTransaction> {
@ -455,8 +412,8 @@ fn get_state_proofs() {
let request_body = encode_request(&request, req_id); let request_body = encode_request(&request, req_id);
let response = { let response = {
let proofs = vec![ let proofs = vec![
{ let mut stream = RlpStream::new_list(2); stream.append(&key1).append_empty_data(); stream.out() }, { let mut stream = RlpStream::new_list(2); stream.append(&key1).append_empty_data(); vec![stream.out()] },
::util::sha3::SHA3_NULL_RLP.to_vec(), vec![::util::sha3::SHA3_NULL_RLP.to_vec()],
]; ];
let new_buf = *flow_params.limit() - flow_params.compute_cost(request::Kind::StateProofs, 2); let new_buf = *flow_params.limit() - flow_params.compute_cost(request::Kind::StateProofs, 2);
@ -465,7 +422,10 @@ fn get_state_proofs() {
response_stream.append(&req_id).append(&new_buf).begin_list(2); response_stream.append(&req_id).append(&new_buf).begin_list(2);
for proof in proofs { for proof in proofs {
response_stream.append_raw(&proof, 1); response_stream.begin_list(proof.len());
for node in proof {
response_stream.append_raw(&node, 1);
}
} }
response_stream.out() response_stream.out()

View File

@ -52,31 +52,139 @@ pub trait Provider: Send + Sync {
/// ///
/// The returned vector may have any length in the range [0, `max`], but the /// The returned vector may have any length in the range [0, `max`], but the
/// results within must adhere to the `skip` and `reverse` parameters. /// results within must adhere to the `skip` and `reverse` parameters.
fn block_headers(&self, req: request::Headers) -> Vec<Bytes>; fn block_headers(&self, req: request::Headers) -> Vec<Bytes> {
use request::HashOrNumber;
use ethcore::views::HeaderView;
if req.max == 0 { return Vec::new() }
let best_num = self.chain_info().best_block_number;
let start_num = match req.start {
HashOrNumber::Number(start_num) => start_num,
HashOrNumber::Hash(hash) => match self.block_header(BlockId::Hash(hash)) {
None => {
trace!(target: "les_provider", "Unknown block hash {} requested", hash);
return Vec::new();
}
Some(header) => {
let num = HeaderView::new(&header).number();
let canon_hash = self.block_header(BlockId::Number(num))
.map(|h| HeaderView::new(&h).hash());
if req.max == 1 || canon_hash != Some(hash) {
// Non-canonical header or single header requested.
return vec![header];
}
num
}
}
};
(0u64..req.max as u64)
.map(|x: u64| x.saturating_mul(req.skip + 1))
.take_while(|x| if req.reverse { x < &start_num } else { best_num - start_num >= *x })
.map(|x| if req.reverse { start_num - x } else { start_num + x })
.map(|x| self.block_header(BlockId::Number(x)))
.take_while(|x| x.is_some())
.flat_map(|x| x)
.collect()
}
/// Get a block header by id.
fn block_header(&self, id: BlockId) -> Option<Bytes>;
/// Provide as many as possible of the requested blocks (minus the headers) encoded /// Provide as many as possible of the requested blocks (minus the headers) encoded
/// in RLP format. /// in RLP format.
fn block_bodies(&self, req: request::Bodies) -> Vec<Bytes>; fn block_bodies(&self, req: request::Bodies) -> Vec<Bytes> {
req.block_hashes.into_iter()
.map(|hash| self.block_body(BlockId::Hash(hash)))
.map(|body| body.unwrap_or_else(|| ::rlp::EMPTY_LIST_RLP.to_vec()))
.collect()
}
/// Get a block body by id.
fn block_body(&self, id: BlockId) -> Option<Bytes>;
/// Provide the receipts as many as possible of the requested blocks. /// Provide the receipts as many as possible of the requested blocks.
/// Returns a vector of RLP-encoded lists of receipts. /// Returns a vector of RLP-encoded lists of receipts.
fn receipts(&self, req: request::Receipts) -> Vec<Bytes>; fn receipts(&self, req: request::Receipts) -> Vec<Bytes> {
req.block_hashes.into_iter()
.map(|hash| self.block_receipts(&hash))
.map(|receipts| receipts.unwrap_or_else(|| ::rlp::EMPTY_LIST_RLP.to_vec()))
.collect()
}
/// Get a block's receipts as an RLP-encoded list by block hash.
fn block_receipts(&self, hash: &H256) -> Option<Bytes>;
/// Provide a set of merkle proofs, as requested. Each request is a /// Provide a set of merkle proofs, as requested. Each request is a
/// block hash and request parameters. /// block hash and request parameters.
/// ///
/// Returns a vector of RLP-encoded lists satisfying the requests. /// Returns a vector of RLP-encoded lists satisfying the requests.
fn proofs(&self, req: request::StateProofs) -> Vec<Bytes>; fn proofs(&self, req: request::StateProofs) -> Vec<Bytes> {
use rlp::{RlpStream, Stream};
let mut results = Vec::with_capacity(req.requests.len());
for request in req.requests {
let proof = self.state_proof(request);
let mut stream = RlpStream::new_list(proof.len());
for node in proof {
stream.append_raw(&node, 1);
}
results.push(stream.out());
}
results
}
/// Get a state proof from a request. Each proof should be a vector
/// of rlp-encoded trie nodes, in ascending order by distance from the root.
fn state_proof(&self, req: request::StateProof) -> Vec<Bytes>;
/// Provide contract code for the specified (block_hash, account_hash) pairs. /// Provide contract code for the specified (block_hash, account_hash) pairs.
/// Each item in the resulting vector is either the raw bytecode or empty. /// Each item in the resulting vector is either the raw bytecode or empty.
fn contract_code(&self, req: request::ContractCodes) -> Vec<Bytes>; fn contract_codes(&self, req: request::ContractCodes) -> Vec<Bytes> {
req.code_requests.into_iter()
.map(|req| self.contract_code(req))
.collect()
}
/// Provide header proofs from the Canonical Hash Tries as well as the headers /// Get contract code by request. Either the raw bytecode or empty.
fn contract_code(&self, req: request::ContractCode) -> Bytes;
/// Provide header proofs from the Canonical Hash Tries as well as the headers
/// they correspond to -- each element in the returned vector is a 2-tuple. /// they correspond to -- each element in the returned vector is a 2-tuple.
/// The first element is a block header and the second a merkle proof of /// The first element is a block header and the second a merkle proof of
/// the header in a requested CHT. /// the header in a requested CHT.
fn header_proofs(&self, req: request::HeaderProofs) -> Vec<Bytes>; fn header_proofs(&self, req: request::HeaderProofs) -> Vec<Bytes> {
use rlp::{self, RlpStream, Stream};
req.requests.into_iter()
.map(|req| self.header_proof(req))
.map(|maybe_proof| match maybe_proof {
None => rlp::EMPTY_LIST_RLP.to_vec(),
Some((header, proof)) => {
let mut stream = RlpStream::new_list(2);
stream.append_raw(&header, 1).begin_list(proof.len());
for node in proof {
stream.append_raw(&node, 1);
}
stream.out()
}
})
.collect()
}
/// Provide a header proof from a given Canonical Hash Trie as well as the
/// corresponding header. The first element is the block header and the
/// second is a merkle proof of the CHT.
fn header_proof(&self, req: request::HeaderProof) -> Option<(Bytes, Vec<Bytes>)>;
/// Provide pending transactions. /// Provide pending transactions.
fn ready_transactions(&self) -> Vec<PendingTransaction>; fn ready_transactions(&self) -> Vec<PendingTransaction>;
@ -96,86 +204,31 @@ impl<T: ProvingBlockChainClient + ?Sized> Provider for T {
Some(self.pruning_info().earliest_state) Some(self.pruning_info().earliest_state)
} }
fn block_headers(&self, req: request::Headers) -> Vec<Bytes> { fn block_header(&self, id: BlockId) -> Option<Bytes> {
use request::HashOrNumber; BlockChainClient::block_header(self, id)
use ethcore::views::HeaderView;
let best_num = self.chain_info().best_block_number;
let start_num = match req.start {
HashOrNumber::Number(start_num) => start_num,
HashOrNumber::Hash(hash) => match self.block_header(BlockId::Hash(hash)) {
None => {
trace!(target: "les_provider", "Unknown block hash {} requested", hash);
return Vec::new();
}
Some(header) => {
let num = HeaderView::new(&header).number();
if req.max == 1 || self.block_hash(BlockId::Number(num)) != Some(hash) {
// Non-canonical header or single header requested.
return vec![header];
}
num
}
}
};
(0u64..req.max as u64)
.map(|x: u64| x.saturating_mul(req.skip + 1))
.take_while(|x| if req.reverse { x < &start_num } else { best_num - start_num >= *x })
.map(|x| if req.reverse { start_num - x } else { start_num + x })
.map(|x| self.block_header(BlockId::Number(x)))
.take_while(|x| x.is_some())
.flat_map(|x| x)
.collect()
} }
fn block_bodies(&self, req: request::Bodies) -> Vec<Bytes> { fn block_body(&self, id: BlockId) -> Option<Bytes> {
req.block_hashes.into_iter() BlockChainClient::block_body(self, id)
.map(|hash| self.block_body(BlockId::Hash(hash)))
.map(|body| body.unwrap_or_else(|| ::rlp::EMPTY_LIST_RLP.to_vec()))
.collect()
} }
fn receipts(&self, req: request::Receipts) -> Vec<Bytes> { fn block_receipts(&self, hash: &H256) -> Option<Bytes> {
req.block_hashes.into_iter() BlockChainClient::block_receipts(self, hash)
.map(|hash| self.block_receipts(&hash))
.map(|receipts| receipts.unwrap_or_else(|| ::rlp::EMPTY_LIST_RLP.to_vec()))
.collect()
} }
fn proofs(&self, req: request::StateProofs) -> Vec<Bytes> { fn state_proof(&self, req: request::StateProof) -> Vec<Bytes> {
use rlp::{RlpStream, Stream}; match req.key2 {
Some(key2) => self.prove_storage(req.key1, key2, req.from_level, BlockId::Hash(req.block)),
let mut results = Vec::with_capacity(req.requests.len()); None => self.prove_account(req.key1, req.from_level, BlockId::Hash(req.block)),
for request in req.requests {
let proof = match request.key2 {
Some(key2) => self.prove_storage(request.key1, key2, request.from_level, BlockId::Hash(request.block)),
None => self.prove_account(request.key1, request.from_level, BlockId::Hash(request.block)),
};
let mut stream = RlpStream::new_list(proof.len());
for node in proof {
stream.append_raw(&node, 1);
}
results.push(stream.out());
} }
results
} }
fn contract_code(&self, req: request::ContractCodes) -> Vec<Bytes> { fn contract_code(&self, req: request::ContractCode) -> Bytes {
req.code_requests.into_iter() self.code_by_hash(req.account_key, BlockId::Hash(req.block_hash))
.map(|req| {
self.code_by_hash(req.account_key, BlockId::Hash(req.block_hash))
})
.collect()
} }
fn header_proofs(&self, req: request::HeaderProofs) -> Vec<Bytes> { fn header_proof(&self, _req: request::HeaderProof) -> Option<(Bytes, Vec<Bytes>)> {
req.requests.into_iter().map(|_| ::rlp::EMPTY_LIST_RLP.to_vec()).collect() None
} }
fn ready_transactions(&self) -> Vec<PendingTransaction> { fn ready_transactions(&self) -> Vec<PendingTransaction> {