Shell: A bit of randomization for gap in block locators.

The random seed is computed from the sender and receiver `peer_id`.
This commit is contained in:
Pietro 2018-03-21 15:38:41 +01:00 committed by Benjamin Canou
parent 5f69ae87d2
commit 80cdf6d0b3
15 changed files with 275 additions and 275 deletions

View File

@ -10,8 +10,6 @@
open Lwt.Infix
type t = raw
(** Non private version of Block_store_locator.t for coercions *)
and raw = Block_header.t * Block_hash.t list
let raw x = x
@ -27,7 +25,8 @@ let pp ppf (hd, h_lst) =
| hd :: tl ->
let new_d = if r > 1 then d else d * coef in
let new_r = if r > 1 then r - 1 else repeats in
Format.fprintf ppf "%a (%i)\n%a" Block_hash.pp hd acc pp_hash_list (tl , acc - d , new_d , new_r) in
Format.fprintf ppf "%a (%i)\n%a"
Block_hash.pp hd acc pp_hash_list (tl , acc - d , new_d , new_r) in
Format.fprintf ppf "%a (head)\n%a"
Block_hash.pp (Block_header.hash hd)
pp_hash_list (h_lst , -1, 1, repeats - 1)
@ -44,50 +43,130 @@ let encoding =
(req "current_head" (dynamic_size Block_header.encoding))
(req "history" (dynamic_size (list Block_hash.encoding))))
(** Computes a locator for block [b] picking 10 times the immediate
predecessors of [b], then 10 times one predecessor every 2, then
10 times one predecessor every 4, ..., until genesis or it reaches
the desired size. *)
let compute ~predecessor ~genesis b header size =
if size < 0 then
invalid_arg "Block_locator.compute: negative size"
else
let repeats = 10 in (* number of repetitions for each power of 2 *)
let rec loop acc size step cnt b =
if size = 0 then
Lwt.return (List.rev acc)
type seed = {
sender_id: P2p_peer.Id.t ;
receiver_id: P2p_peer.Id.t ;
}
(* Random generator for locator steps.
We draw steps by sequence of 10. The first sequence's steps are of
length 1 (consecutive). The second sequence's steps are of a random
length between 1 and 2. The third sequence's steps are of a random
length between 2 and 4, and so on...
The sequence is deterministic for a given triple of sender,
receiver and block hash. *)
module Step : sig
type state
val init: seed -> Block_hash.t -> state
val next: state -> int * state
end = struct
type state = int * int * Cstruct.t
let init seed head =
let seed =
Nocrypto.Hash.digest `SHA256 @@
Cstruct.concat
[ Cstruct.of_bigarray @@ P2p_peer.Id.to_bytes seed.sender_id ;
Cstruct.of_bigarray @@ P2p_peer.Id.to_bytes seed.receiver_id ;
Cstruct.of_bigarray @@ Block_hash.to_bytes head ] in
(1, 9, seed)
let draw seed n =
Int32.to_int (MBytes.get_int32 (Cstruct.to_bigarray seed) 0) mod n,
Nocrypto.Hash.digest `SHA256 seed
let next (step, counter, seed) =
let random_gap, seed =
if step <= 1 then
0, seed
else
predecessor b step >>= function
| None -> (* reached genesis before size *)
if Block_hash.equal b genesis then
Lwt.return (List.rev acc)
else
Lwt.return (List.rev (genesis :: acc))
| Some pred ->
if cnt = 1 then
loop (pred :: acc) (size - 1) (step * 2) repeats pred
else
loop (pred :: acc) (size - 1) step (cnt - 1) pred in
draw seed (1 + step/2) in
let new_state =
if counter = 0 then
(step * 2, 9, seed)
else
(step, counter - 1, seed) in
step - random_gap, new_state
end
let estimated_length seed (head, hist) =
let rec loop acc state = function
| [] -> acc
| _ :: hist ->
let step, state = Step.next state in
loop (acc + step) state hist in
let state = Step.init seed (Block_header.hash head) in
let step, state = Step.next state in
loop step state hist
let fold ~f ~init (head, hist) seed =
let rec loop state acc = function
| [] | [_] -> acc
| block :: (pred :: rem as hist) ->
let step, state = Step.next state in
let acc = f acc ~block ~pred ~step ~strict_step:(rem <> []) in
loop state acc hist in
let head = Block_header.hash head in
let state = Step.init seed head in
loop state init (head :: hist)
type step = {
block: Block_hash.t ;
predecessor: Block_hash.t ;
step: int ;
strict_step: bool ;
}
let to_steps seed locator =
fold locator seed
~init:[]
~f: begin fun acc ~block ~pred ~step ~strict_step ->
{ block ; predecessor = pred ; step ; strict_step } :: acc
end
let compute ~predecessor ~genesis block_hash header seed ~size =
let rec loop acc size state block =
if size = 0 then
Lwt.return (header, [])
Lwt.return (List.rev acc)
else
predecessor b 1 >>= function
| None -> Lwt.return (header, [])
| Some p ->
loop [p] (size-1) 1 (repeats-1) p >>= fun hist ->
Lwt.return (header, hist)
let step, state = Step.next state in
predecessor block step >>= function
| None ->
(* We reached genesis before size *)
if Block_hash.equal block genesis then
Lwt.return (List.rev acc)
else
Lwt.return (List.rev (genesis :: acc))
| Some pred ->
loop (pred :: acc) (size - 1) state pred in
if size <= 0 then
Lwt.return (header, [])
else
let state = Step.init seed block_hash in
let step, state = Step.next state in
predecessor block_hash step >>= function
| None -> Lwt.return (header, [])
| Some p ->
loop [p] (size-1) state p >>= fun hist ->
Lwt.return (header, hist)
type validity =
| Unknown
| Known_valid
| Known_invalid
let unknown_prefix cond (head, hist) =
let unknown_prefix ~is_known (head, hist) =
let rec loop hist acc =
match hist with
| [] -> Lwt.return_none
| h :: t ->
cond h >>= function
is_known h >>= function
| Known_valid ->
Lwt.return_some (h, (List.rev (h :: acc)))
| Known_invalid ->
@ -95,7 +174,7 @@ let unknown_prefix cond (head, hist) =
| Unknown ->
loop t (h :: acc)
in
cond (Block_header.hash head) >>= function
is_known (Block_header.hash head) >>= function
| Known_valid ->
Lwt.return_some (Block_header.hash head, (head, []))
| Known_invalid ->

View File

@ -14,17 +14,43 @@ and raw = Block_header.t * Block_hash.t list
(** Non private version of Block_store_locator.t for coercions *)
val raw: t -> raw
val pp : Format.formatter -> t -> unit
val pp_short : Format.formatter -> t -> unit
val pp: Format.formatter -> t -> unit
val pp_short: Format.formatter -> t -> unit
val encoding: t Data_encoding.t
val compute: predecessor: (Block_hash.t -> int -> Block_hash.t option Lwt.t) ->
genesis:Block_hash.t -> Block_hash.t -> Block_header.t -> int -> t Lwt.t
(** [compute block max_length] compute the sparse block locator for
the [block]. The locator contains at most [max_length] elements. *)
type seed = {
sender_id: P2p_peer.Id.t ;
receiver_id: P2p_peer.Id.t
}
(** Argument to the seed used to randomize the locator. *)
val estimated_length: seed -> t -> int
(** [estimated_length seed locator] estimate the length of the chain
represented by [locator] using [seed]. *)
val compute:
predecessor: (Block_hash.t -> int -> Block_hash.t option Lwt.t) ->
genesis:Block_hash.t ->
Block_hash.t -> Block_header.t -> seed -> size:int -> t Lwt.t
(** [compute block seed max_length] compute the sparse block locator using
[seed] to compute random jumps for the [block]. The locator contains at
most [max_length] elements. *)
type step = {
block: Block_hash.t ;
predecessor: Block_hash.t ;
step: int ;
strict_step: bool ;
}
(** A 'step' in a locator is a couple of consecutive hashes in the
locator, and the expected difference of level between the two
blocks (or an upper bounds when [strict_step = false]). *)
val to_steps: seed -> t -> step list
(** Build all the 'steps' composing the locator using a given seed,
starting with the oldest one (typically the predecessor of the
first step will be `genesis`).
All steps contains [strict_step = true], except the oldest one. *)
type validity =
| Unknown
@ -32,7 +58,8 @@ type validity =
| Known_invalid
val unknown_prefix:
(Block_hash.t -> validity Lwt.t) -> t -> (Block_hash.t * t) option Lwt.t
is_known:(Block_hash.t -> validity Lwt.t) ->
t -> (Block_hash.t * t) option Lwt.t
(** [unknown_prefix validity locator] keeps only the unknown part of
the locator up to the first valid block. If there is no known valid
block or there is a known invalid one, None is returned. *)

View File

@ -1,93 +0,0 @@
(**************************************************************************)
(* *)
(* Copyright (c) 2014 - 2018. *)
(* Dynamic Ledger Solutions, Inc. <contact@tezos.com> *)
(* *)
(* All rights reserved. No warranty, explicit or implicit, provided. *)
(* *)
(**************************************************************************)
let estimated_length locator =
let (_head, hist) = Block_locator.raw locator in
let rec loop acc step cpt = function
| [] -> acc
| _ :: hist ->
if cpt = 0 then
loop (acc+step) (step*2) 9 hist
else
loop (acc+step) step (cpt-1) hist
in
loop 1 1 9 hist
let fold ~f acc locator =
let (head, hist) = Block_locator.raw locator in
let rec loop step cpt acc = function
| [] | [_] -> acc
| block :: (pred :: rem as hist) ->
let step, cpt =
if cpt = 0 then
2 * step, 9
else
step, cpt - 1 in
let acc = f acc ~block ~pred ~step ~strict_step:(rem <> []) in
loop step cpt acc hist
in
loop 1 10 acc (Block_header.hash head :: hist)
type step = {
block: Block_hash.t ;
predecessor: Block_hash.t ;
step: int ;
strict_step: bool ;
}
let to_steps locator =
fold
~f:begin fun acc ~block ~pred ~step ~strict_step -> {
block ; predecessor = pred ; step ; strict_step ;
} :: acc
end
[] locator
let block_validity chain_state block : Block_locator.validity Lwt.t =
State.Block.known chain_state block >>= function
| false ->
if Block_hash.equal block (State.Chain.faked_genesis_hash chain_state) then
Lwt.return Block_locator.Known_valid
else
Lwt.return Block_locator.Unknown
| true ->
State.Block.known_invalid chain_state block >>= function
| true ->
Lwt.return Block_locator.Known_invalid
| false ->
Lwt.return Block_locator.Known_valid
let known_ancestor chain_state locator =
Block_locator.unknown_prefix (block_validity chain_state) locator >>= function
| None -> Lwt.return_none
| Some (tail, locator) ->
if Block_hash.equal tail (State.Chain.faked_genesis_hash chain_state) then
State.Block.read_exn
chain_state (State.Chain.genesis chain_state).block >>= fun genesis ->
Lwt.return_some (genesis, locator)
else
State.Block.read_exn chain_state tail >>= fun block ->
Lwt.return_some (block, locator)
let find_new chain_state locator sz =
let rec path sz acc h =
if sz <= 0 then Lwt.return (List.rev acc)
else
State.read_chain_data chain_state begin fun chain_store _data ->
Store.Chain_data.In_main_branch.read_opt (chain_store, h)
end >>= function
| None -> Lwt.return (List.rev acc)
| Some s -> path (sz-1) (s :: acc) s in
known_ancestor chain_state locator >>= function
| None -> Lwt.return_nil
| Some (known, _) ->
Chain.head chain_state >>= fun head ->
Chain_traversal.common_ancestor known head >>= fun ancestor ->
path sz [] (State.Block.hash ancestor)

View File

@ -1,55 +0,0 @@
(**************************************************************************)
(* *)
(* Copyright (c) 2014 - 2018. *)
(* Dynamic Ledger Solutions, Inc. <contact@tezos.com> *)
(* *)
(* All rights reserved. No warranty, explicit or implicit, provided. *)
(* *)
(**************************************************************************)
val fold:
f:('a ->
block:Block_hash.t -> pred:Block_hash.t ->
step:int -> strict_step:bool -> 'a) ->
'a -> Block_locator.t -> 'a
(** [map f l] applies [f] to each block of the locator, the last one
excepted. The function also receives the expected predecessor
[pred] of the [block] after [step] steps, i.e. the next block in
the locator. When [strict_step] is [true], then [step] is the
exact number of predecessor to be followed before to found
[pred]. Otherwise, it is only an upper bound. *)
type step = {
block: Block_hash.t ;
predecessor: Block_hash.t ;
step: int ;
strict_step: bool ;
}
(** A 'step' in a locator is a couple of consecutives hashes in the
locator, and the expected difference of level the two blocks (or
an upper bounds when [strict_step = false]). *)
val to_steps: Block_locator.t -> step list
(** Build all the 'steps' composing the locator, starting with the
oldest one (typically the predecessor of the first step will be
`genesis`). All steps contains [strict_step = true], except the
first one. *)
val estimated_length: Block_locator.t -> int
(** [estimated_length locator] estimate the length of the chain
represented by [locator]. *)
val known_ancestor:
State.Chain.t -> Block_locator.t -> (State.Block.t * Block_locator.t) option Lwt.t
(** [known_ancestor chain_state locator] computes the first block of
[locator] that is known to be a valid block. It also computes the
'prefix' of [locator] with end at the first valid block. The
function returns [None] when no block in the locator are known or
if the first known block is invalid. *)
val find_new:
State.Chain.t -> Block_locator.t -> int -> Block_hash.t list Lwt.t
(** [find_new chain locator max_length] returns the blocks from our
current branch that would be unknown to a peer that sends us the
[locator]. *)

View File

@ -31,7 +31,7 @@ type t = {
mutable errors: Error_monad.error list ;
}
let fetch_step pipeline (step : Block_locator_iterator.step) =
let fetch_step pipeline (step : Block_locator.step) =
lwt_log_info "fetching step %a -> %a (%d%s) from peer %a."
Block_hash.pp_short step.block
Block_hash.pp_short step.predecessor
@ -78,7 +78,11 @@ let fetch_step pipeline (step : Block_locator_iterator.step) =
let headers_fetch_worker_loop pipeline =
begin
let steps = Block_locator_iterator.to_steps pipeline.locator in
let sender_id = Distributed_db.my_peer_id pipeline.chain_db in
(* sender and receiver are inverted here because they are from
the point of view of the node sending the locator *)
let seed = {Block_locator.sender_id=pipeline.peer_id; receiver_id=sender_id } in
let steps = Block_locator.to_steps seed pipeline.locator in
iter_s (fetch_step pipeline) steps >>=? fun () ->
return ()
end >>= function

View File

@ -39,7 +39,6 @@ type data = State.chain_data = {
current_mempool: Mempool.t ;
live_blocks: Block_hash.Set.t ;
live_operations: Operation_hash.Set.t ;
locator: Block_locator.t Lwt.t lazy_t ;
}
let data chain_state =
@ -47,12 +46,11 @@ let data chain_state =
Lwt.return data
end
let locator chain_state =
data chain_state >>= begin fun data ->
Lazy.force data.locator
end
let locator chain_state seed =
data chain_state >>= fun data ->
State.compute_locator chain_state data.current_head seed
let locked_set_head chain_state chain_store data block =
let locked_set_head chain_store data block =
let rec pop_blocks ancestor block =
let hash = State.Block.hash block in
if Block_hash.equal hash ancestor then
@ -89,12 +87,11 @@ let locked_set_head chain_state chain_store data block =
current_mempool = Mempool.empty ;
live_blocks ;
live_operations ;
locator = lazy (State.compute_locator chain_state block) ;
}
let set_head chain_state block =
State.update_chain_data chain_state begin fun chain_store data ->
locked_set_head chain_state chain_store data block >>= fun new_chain_data ->
locked_set_head chain_store data block >>= fun new_chain_data ->
Lwt.return (Some new_chain_data,
data.current_head)
end
@ -104,7 +101,7 @@ let test_and_set_head chain_state ~old block =
if not (State.Block.equal data.current_head old) then
Lwt.return (None, false)
else
locked_set_head chain_state chain_store data block >>= fun new_chain_data ->
locked_set_head chain_store data block >>= fun new_chain_data ->
Lwt.return (Some new_chain_data, true)
end

View File

@ -15,7 +15,7 @@ val genesis: State.Chain.t -> State.Block.t Lwt.t
(** The current head of the chain. *)
val head: State.Chain.t -> State.Block.t Lwt.t
val locator: State.Chain.t -> Block_locator.t Lwt.t
val locator: State.Chain.t -> Block_locator.seed -> Block_locator.t Lwt.t
(** All the available chain data. *)
type data = {
@ -23,13 +23,11 @@ type data = {
current_mempool: Mempool.t ;
live_blocks: Block_hash.Set.t ;
live_operations: Operation_hash.Set.t ;
locator: Block_locator.t Lwt.t lazy_t ;
}
(** Reading atomically all the chain data. *)
val data: State.Chain.t -> data Lwt.t
(** The current head and all the known (valid) alternate heads. *)
val known_heads: State.Chain.t -> State.Block.t list Lwt.t

View File

@ -216,10 +216,7 @@ let broadcast_head w ~previous block =
nv.parameters.chain_db block ;
Lwt.return_unit
end else begin
let chain_state = Distributed_db.chain_state nv.parameters.chain_db in
Chain.locator chain_state >>= fun locator ->
Distributed_db.Advertise.current_branch
nv.parameters.chain_db locator
Distributed_db.Advertise.current_branch nv.parameters.chain_db
end
end

View File

@ -340,6 +340,8 @@ let state { disk } = disk
let chain_state { chain_state } = chain_state
let db { global_db } = global_db
let my_peer_id chain_db = P2p.peer_id chain_db.global_db.p2p
let read_block_header { disk } h =
State.read_block disk h >>= function
| Some b ->
@ -466,7 +468,10 @@ module P2p_reader = struct
ignore
@@ P2p.try_send global_db.p2p state.conn
@@ Get_current_branch chain_id ;
Chain.locator chain_db.chain_state >>= fun locator ->
let seed = {
Block_locator.receiver_id=state.gid;
sender_id=(my_peer_id chain_db) } in
(Chain.locator chain_db.chain_state seed) >>= fun locator ->
ignore
@@ P2p.try_send global_db.p2p state.conn
@@ Current_branch (chain_id, locator) ;
@ -681,6 +686,7 @@ let raw_try_send p2p peer_id msg =
| None -> ()
| Some conn -> ignore (P2p.try_send p2p conn msg : bool)
let create disk p2p =
let global_request =
{ data = () ;
@ -924,10 +930,28 @@ module Advertise = struct
send chain_db ?peer @@
Current_head (chain_id, State.Block.header head, mempool)
let current_branch chain_db ?peer locator =
let current_branch ?peer chain_db =
let chain_id = State.Chain.id chain_db.chain_state in
send chain_db ?peer @@ Current_branch (chain_id, locator) ;
Lwt.return_unit
let chain_state = chain_state chain_db in
let sender_id = my_peer_id chain_db in
match peer with
| Some receiver_id ->
let seed = {
Block_locator.receiver_id=receiver_id; sender_id } in
(Chain.locator chain_state seed) >>= fun locator ->
let msg = Message.Current_branch (chain_id, locator) in
try_send chain_db receiver_id msg;
Lwt.return_unit
| None ->
Lwt_list.iter_p
(fun (receiver_id,state) ->
let seed = {
Block_locator.receiver_id=receiver_id; sender_id } in
(Chain.locator chain_state seed) >>= fun locator ->
let msg = Message.Current_branch (chain_id, locator) in
ignore (P2p.try_send chain_db.global_db.p2p state.conn msg);
Lwt.return_unit
) (P2p_peer.Table.fold (fun k v acc -> (k,v)::acc) chain_db.active_connections [])
end

View File

@ -58,6 +58,9 @@ val disconnect: chain_db -> P2p_peer.Id.t -> unit Lwt.t
val chain_state: chain_db -> State.Chain.t
val db: chain_db -> db
(** Return the peer id of the node *)
val my_peer_id: chain_db -> P2p_peer.Id.t
(** {1 Sending messages} *)
module Request : sig
@ -85,8 +88,7 @@ module Advertise : sig
(** Notify a given peer, or all known active peers for the
chain, of a new head and its sparse history. *)
val current_branch:
chain_db -> ?peer:P2p_peer.Id.t ->
Block_locator.t -> unit Lwt.t
?peer:P2p_peer.Id.t -> chain_db -> unit Lwt.t
end

View File

@ -26,13 +26,15 @@ module Request = struct
type _ t =
| New_head : Block_hash.t * Block_header.t -> unit t
| New_branch : Block_hash.t * Block_locator.t -> unit t
| New_branch : Block_hash.t * Block_locator.t * Block_locator.seed -> unit t
let view (type a) (req : a t) : view = match req with
| New_head (hash, _) ->
New_head hash
| New_branch (hash, locator) ->
New_branch (hash, Block_locator_iterator.estimated_length locator)
| New_branch (hash, locator, seed) ->
(* the seed is associated to each locator
w.r.t. the peer_id of the sender *)
New_branch (hash, Block_locator.estimated_length seed locator)
end
type limits = {
@ -89,7 +91,11 @@ let set_bootstrapped pv =
let bootstrap_new_branch w _ancestor _head unknown_prefix =
let pv = Worker.state w in
let len = Block_locator_iterator.estimated_length unknown_prefix in
let sender_id = Distributed_db.my_peer_id pv.parameters.chain_db in
(* sender and receiver are inverted here because they are from
the point of view of the node sending the locator *)
let seed = {Block_locator.sender_id=pv.peer_id; receiver_id = sender_id } in
let len = Block_locator.estimated_length seed unknown_prefix in
debug w
"validating new branch from peer %a (approx. %d blocks)"
P2p_peer.Id.pp_short pv.peer_id len ;
@ -201,7 +207,7 @@ let may_validate_new_branch w distant_hash locator =
let distant_header, _ = (locator : Block_locator.t :> Block_header.t * _) in
only_if_fitness_increases w distant_header @@ fun () ->
let chain_state = Distributed_db.chain_state pv.parameters.chain_db in
Block_locator_iterator.known_ancestor chain_state locator >>= function
State.Block.known_ancestor chain_state locator >>= function
| None ->
debug w
"ignoring branch %a without common ancestor from peer: %a."
@ -228,7 +234,7 @@ let on_request (type a) w (req : a Request.t) : a tzresult Lwt.t =
Block_hash.pp_short hash
P2p_peer.Id.pp_short pv.peer_id ;
may_validate_new_head w hash header
| Request.New_branch (hash, locator) ->
| Request.New_branch (hash, locator, _seed) ->
(* TODO penalize empty locator... ?? *)
debug w "processing new branch %a from peer %a."
Block_hash.pp_short hash
@ -300,7 +306,7 @@ let table =
let merge w (Worker.Any_request neu) old =
let pv = Worker.state w in
match neu with
| Request.New_branch (_, locator) ->
| Request.New_branch (_, locator, _) ->
let header, _ = (locator : Block_locator.t :> _ * _) in
pv.last_advertised_head <- header ;
Some (Worker.Any_request neu)
@ -346,7 +352,12 @@ let create
let notify_branch w locator =
let header, _ = (locator : Block_locator.t :> _ * _) in
let hash = Block_header.hash header in
Worker.drop_request w (New_branch (hash, locator))
let pv = Worker.state w in
let sender_id = Distributed_db.my_peer_id pv.parameters.chain_db in
(* sender and receiver are inverted here because they are from
the point of view of the node sending the locator *)
let seed = {Block_locator.sender_id=pv.peer_id; receiver_id=sender_id } in
Worker.drop_request w (New_branch (hash, locator, seed))
let notify_head w header =
let hash = Block_header.hash header in

View File

@ -60,7 +60,6 @@ and chain_data = {
current_mempool: Mempool.t ;
live_blocks: Block_hash.Set.t ;
live_operations: Operation_hash.Set.t ;
locator: Block_locator.t Lwt.t lazy_t ;
}
and block = {
@ -139,7 +138,7 @@ let store_predecessors (store: Store.Block.store) (b: Block_hash.t) : unit Lwt.t
and in O(|chain|) after that.
@raise Invalid_argument "State.predecessors: negative distance"
*)
let predecessor_n (store: Store.Block.store) (b: Block_hash.t) (distance: int)
let predecessor_n (store: Store.Block.store) (block_hash: Block_hash.t) (distance: int)
: Block_hash.t option Lwt.t =
(* helper functions *)
(* computes power of 2 w/o floats *)
@ -167,9 +166,9 @@ let predecessor_n (store: Store.Block.store) (b: Block_hash.t) (distance: int)
if distance <= 0 then
invalid_arg ("State.predecessor: distance <= 0"^(string_of_int distance))
else
let rec loop b distance =
let rec loop block_hash distance =
if distance = 1
then Store.Block.Predecessors.read_opt (store, b) 0
then Store.Block.Predecessors.read_opt (store, block_hash) 0
else
let (power,rest) = closest_power_two_and_rest distance in
let (power,rest) =
@ -179,24 +178,24 @@ let predecessor_n (store: Store.Block.store) (b: Block_hash.t) (distance: int)
let rest = distance - (power_of_2 power) in
(power,rest)
in
Store.Block.Predecessors.read_opt (store, b) power >>= function
Store.Block.Predecessors.read_opt (store, block_hash) power >>= function
| None -> Lwt.return_none (* reached genesis *)
| Some pred ->
if rest = 0
then Lwt.return_some pred (* landed on the requested predecessor *)
else loop pred rest (* need to jump further back *)
in
loop b distance
loop block_hash distance
let compute_locator_from_hash (chain : chain_state) ?(size = 200) head_hash =
let compute_locator_from_hash (chain : chain_state) ?(size = 200) head_hash seed =
Shared.use chain.block_store begin fun block_store ->
Store.Block.Contents.read_exn (block_store, head_hash) >>= fun { header } ->
Block_locator.compute ~predecessor:(predecessor_n block_store)
~genesis:chain.genesis.block head_hash header size
~genesis:chain.genesis.block head_hash header seed ~size
end
let compute_locator chain ?size head =
compute_locator_from_hash chain ?size head.hash
let compute_locator chain ?size head seed =
compute_locator_from_hash chain ?size head.hash seed
type t = global_state
@ -259,7 +258,6 @@ module Chain = struct
current_mempool = Mempool.empty ;
live_blocks = Block_hash.Set.singleton genesis.block ;
live_operations = Operation_hash.Set.empty ;
locator = lazy (compute_locator_from_hash chain_state current_head) ;
} ;
chain_data_store ;
}
@ -645,6 +643,33 @@ module Block = struct
context block >>= fun context ->
Context.get_test_chain context
let block_validity chain_state block : Block_locator.validity Lwt.t =
known chain_state block >>= function
| false ->
if Block_hash.equal block (Chain.faked_genesis_hash chain_state) then
Lwt.return Block_locator.Known_valid
else
Lwt.return Block_locator.Unknown
| true ->
known_invalid chain_state block >>= function
| true ->
Lwt.return Block_locator.Known_invalid
| false ->
Lwt.return Block_locator.Known_valid
let known_ancestor chain_state locator =
Block_locator.unknown_prefix
~is_known:(block_validity chain_state) locator >>= function
| None -> Lwt.return_none
| Some (tail, locator) ->
if Block_hash.equal tail (Chain.faked_genesis_hash chain_state) then
read_exn
chain_state (Chain.genesis chain_state).block >>= fun genesis ->
Lwt.return_some (genesis, locator)
else
read_exn chain_state tail >>= fun block ->
Lwt.return_some (block, locator)
end
let read_block { global_data } ?pred hash =

View File

@ -148,6 +148,14 @@ module Block : sig
val watcher: Chain.t -> block Lwt_stream.t * Lwt_watcher.stopper
val known_ancestor:
Chain.t -> Block_locator.t -> (block * Block_locator.t) option Lwt.t
(** [known_ancestor chain_state locator] computes the first block of
[locator] that is known to be a valid block. It also computes the
'prefix' of [locator] with end at the first valid block. The
function returns [None] when no block in the locator are known or
if the first known block is invalid. *)
end
val read_block:
@ -156,7 +164,7 @@ val read_block:
val read_block_exn:
global_state -> ?pred:int -> Block_hash.t -> Block.t Lwt.t
val compute_locator: Chain.t -> ?size:int -> Block.t -> Block_locator.t Lwt.t
val compute_locator: Chain.t -> ?size:int -> Block.t -> Block_locator.seed -> Block_locator.t Lwt.t
val fork_testchain:
Block.t -> Protocol_hash.t -> Time.t -> Chain.t tzresult Lwt.t
@ -166,7 +174,6 @@ type chain_data = {
current_mempool: Mempool.t ;
live_blocks: Block_hash.Set.t ;
live_operations: Operation_hash.Set.t ;
locator: Block_locator.t Lwt.t lazy_t ;
}
val read_chain_data:
@ -217,4 +224,3 @@ module Current_mempool : sig
not the provided one. *)
end

View File

@ -226,6 +226,10 @@ let test_pred (base_dir:string) : unit tzresult Lwt.t =
let repeats = 100 in
return (repeat (fun () -> test_once (1 + Random.int range)) repeats)
let seed =
let receiver_id = P2p_peer.Id.of_string_exn (String.make P2p_peer.Id.size 'r') in
let sender_id = P2p_peer.Id.of_string_exn (String.make P2p_peer.Id.size 's') in
{Block_locator.receiver_id=receiver_id ; sender_id }
(* compute locator using the linear predecessor *)
let compute_linear_locator (chain:State.Chain.t) ~size block =
@ -233,7 +237,7 @@ let compute_linear_locator (chain:State.Chain.t) ~size block =
let block_hash = State.Block.hash block in
let header = State.Block.header block in
Block_locator.compute ~predecessor:(linear_predecessor_n chain)
~genesis:genesis.block block_hash header size
~genesis:genesis.block block_hash header ~size seed
(* given the size of a chain, returns the size required for a locator
@ -284,7 +288,7 @@ let test_locator base_dir =
let check_locator size : unit tzresult Lwt.t =
State.Block.read chain head >>=? fun block ->
time ~runs:runs (fun () ->
State.compute_locator chain ~size:size block) |>
State.compute_locator chain ~size:size block seed) |>
fun (l_exp,t_exp) ->
time ~runs:runs (fun () ->
compute_linear_locator chain ~size:size block) |>
@ -312,7 +316,6 @@ let test_locator base_dir =
in
loop 1
let wrap n f =
Alcotest_lwt.test_case n `Quick begin fun _ () ->
Lwt_utils_unix.with_tempdir "tezos_test_" begin fun dir ->

View File

@ -234,12 +234,17 @@ let test_ancestor s =
(****************************************************************************)
let seed =
let receiver_id = P2p_peer.Id.of_string_exn (String.make P2p_peer.Id.size 'r') in
let sender_id = P2p_peer.Id.of_string_exn (String.make P2p_peer.Id.size 's') in
{Block_locator.receiver_id=receiver_id ; sender_id }
(** Chain_traversal.block_locator *)
let test_locator s =
let check_locator length h1 expected =
State.compute_locator s.chain
~size:length (vblock s h1) >>= fun l ->
~size:length (vblock s h1) seed >>= fun l ->
let _, l = (l : Block_locator.t :> _ * _) in
if List.length l <> List.length expected then
Assert.fail_msg
@ -378,35 +383,6 @@ let test_new_blocks s =
return ()
(****************************************************************************)
(** Block_locator.find_new *)
let test_find_new s =
let test s h expected =
State.compute_locator s.chain ~size:50 (vblock s h) >>= fun loc ->
Block_locator_iterator.find_new
s.chain loc (List.length expected) >>= fun blocks ->
if List.length blocks <> List.length expected then
Assert.fail_msg
"Invalid find new length %s (found: %d, expected: %d)"
h (List.length blocks) (List.length expected) ;
List.iter2
(fun h1 h2 ->
if not (Block_hash.equal h1 (State.Block.hash @@ vblock s h2)) then
Assert.fail_msg "Invalid find new %s.%d (expected: %s)" h (List.length expected) h2)
blocks expected ;
Lwt.return_unit
in
Chain.set_head s.chain (vblock s "A8") >>= fun _ ->
test s "A6" [] >>= fun () ->
test s "A6" ["A7";"A8"] >>= fun () ->
test s "A6" ["A7"] >>= fun () ->
test s "B4" ["A4"] >>= fun () ->
test s "B7" ["A4";"A5";"A6";"A7"] >>= fun () ->
return ()
(****************************************************************************)
@ -420,7 +396,6 @@ let tests : (string * (state -> unit tzresult Lwt.t)) list = [
"head", test_head ;
"mem", test_mem ;
"new_blocks", test_new_blocks ;
"find_new", test_find_new ;
]
let wrap (n, f) =