This makes the span data more compact (e.g., `msg_as_req{msg=block}`) and
restores the Debug impl for Message to show all of the data contained in the
message. The full message is added as a single event at trace level in the
span to preserve the previous full-inspectability.
As we approach our alpha release we've decided we want to plan ahead for the user bug reports we will eventually receive. One of the bigger issues we foresee is determining exactly what version of the software users are running, and particularly how easy it may or may not be for users to accidentally discard this information when reporting bugs.
To defend against this, we've decided to include the exact git sha for any given build in the compiled artifact. This information will then be re-exported as a span early in the application startup process, so that all logs and error messages should include the sha as their very first span. We've also added this sha as issue metadata for `color-eyre`'s github issue url auto generation feature, which should make sure that the sha is easily available in bug reports we receive, even in the absence of logs.
Co-authored-by: teor <teor@riseup.net>
* warn: if there are no peers at all
* info: if there are no ready peers
* trace: the number of ready and unready peers for every request
Log at most one warn or info log per minute, to avoid flooding the
terminal with log lines. Suppress warn and info logs for the first
minute, while the peer set is starting up.
* implement inbound `FindBlocks`
* Handle inbound peer FindHeaders requests
* handle request before having any chain tip
* Split `find_chain_hashes` into smaller functions
Add a `max_len` argument to support `FindHeaders` requests.
Rewrite the hash collection code to use heights, so we can handle the
`stop` hash and "no intersection" cases correctly.
* Split state height functions into "any chain" and "best chain"
* Rename the best chain block method to `best_block`
* Move fmt utilities to zebra_chain::fmt
* Summarise Debug for some Message variants
Co-authored-by: teor <teor@riseup.net>
Co-authored-by: Jane Lusby <jlusby42@gmail.com>
Closes#1183.
The peer set maintains a preselected ready service that it can use to
perform power-of-two-choices (p2c) routing of requests. Ready services
are stored by key (socket address) in an `IndexMap`, and the preselected
service is represented by an `Option<usize>` indexing that map. This
means that whenever the set of ready services changes (e.g., a service
is removed from the peer set, or a service is taken to be used to
process a request), the preselected index is invalidated. The original
P2C-only implementation maintained this invariant but did not document
it.
The change to inventory-based routing introduced a bug by failing to
maintain this invariant and appropriately invalidate the preselected
index. However, this was only noticeable approximately 1/N of the time
on the next request after an inventory-directed request, so the bug
occurred infrequently. Luckily, the use of `.expect` caused the bug to
be an immediate panic, making it possible to identify by inspecting all
uses of the ready service map.
This change is mostly mechanical, with the exception of the changes to the
`tower-batch` middleware. This middleware was adapted from `tower::buffer`,
and the `tower::buffer` code was changed to implement its own bounded queue,
because Tokio 0.3 removed the `mpsc::Sender::poll_send` method. See
ddc64e8d4d
for more context on the Tower changes. To match Tower as closely as possible
in order to be able to upstream `tower-batch`, those changes are copied from
`tower::Buffer` to `tower-batch`.
Per https://zips.z.cash/zip-0251, nodes compatible with Canopy
activation on mainnet MUST advertise protocol version 170013 or later.
Once Canopy activates on testnet or mainnet, Canopy nodes SHOULD reject
new connections from pre-Canopy nodes, so this also increases the
minimum version.
This change explicitly documents cancellation contracts for our Tower services,
and tries to correct a bug in the implementation of the CheckpointVerifier,
which duplicates information from the state service but did not ensure that it
would be kept in sync.
These messages might be unsolicited, or they might be a response to a
request we already canceled. So don't fail the whole connection, just
drop the message and move on.
We handle request cancellation in two places: before we transition into
the AwaitingResponse state, and while we are in AwaitingResponse. We
need both places, or else if we started processing a request, we
wouldn't process the cancellation until the timeout elapsed.
The first is a check that the oneshot is not already canceled.
For the second, we wait on a cancellation, either from a timeout or from
the tx channel closing.
This addresses at least three pain points:
- we were affected by bugs that were already fixed in git, but not in
the released crate;
- we can use service combinators to transform requests and responses;
- we can use the hedge middleware.
The version in git is still marked as 0.3.1 but these changes will be
part of tower 0.4: https://github.com/tower-rs/tower/issues/431
The GetPeers requests sent while crawling the network are randomly
load-balanced over available peers. But at the very beginning, they may
be both routed to the same peer, causing network initialization to be
delayed while the second one times out (since zcashd only ever responds
to the first addr message).
Only sending one GetPeers request per candidate set update means we
crawl the network a little more slowly, but avoids hanging on start.
This cleans up the response processing logic a little bit along the way,
but the overall division of responsibility should be better documented
in a future commit.
This lets us distinguish between cases where the message was unsupported
(e.g., BIP11 messages), and cases where the message was uninterpretable
in context (e.g., unsolicited messages).
> Added a test that the handshake's version message matches specified fields, but the test does not compile, because rustc doesn't believe that the Box<dyn std::error::Error + Send + Sync + 'static> is 'static, and therefore isn't a Box<dyn std::error::Error + Send + Sync + 'static>. This manifests as being unable to spawn the connect_isolated task. From digging through Tokio issues I believe that this is an instance of rust-lang/rust#64552 .
Co-authored-by: Jane Lusby <jlusby42@gmail.com>
The peer set provides an automatically managed connection pool, abstracting
away all the details of handling individual peer connections. However, it's
also useful to be able to create completely isolated and
minimally-distinguishable connections to individual peers, in order to be able
to send specific messages over Tor, or to implement some custom network crawler
logic.
* increase the EWMA default and decay
* increase the block download retries
* increase the request and block download timeouts
* increase the sync timeout
This commit makes several related changes to the network code:
- adds a `TransactionsByHash(HashSet<transaction::Hash>)` request and
`Transactions(Vec<Arc<Transaction>>)` response pair that allows
fetching transactions from a remote peer;
- adds a `PushTransaction(Arc<Transaction>)` request that pushes an
unsolicited transaction to a remote peer;
- adds an `AdvertiseTransactions(HashSet<transaction::Hash>)` request
that advertises transactions by hash to a remote peer;
- adds an `AdvertiseBlock(block::Hash)` request that advertises a block
by hash to a remote peer;
Then, it modifies the connection state machine so that outbound
requests to remote peers are handled properly:
- `TransactionsByHash` generates a `getdata` message and collects the
results, like the existing `BlocksByHash` request.
- `PushTransaction` generates a `tx` message, and returns `Nil` immediately.
- `AdvertiseTransactions` and `AdvertiseBlock` generate an `inv`
message, and return `Nil` immediately.
Next, it modifies the connection state machine so that messages
from remote peers generate requests to the inbound service:
- `getdata` messages generate `BlocksByHash` or `TransactionsByHash`
requests, depending on the content of the message;
- `tx` messages generate `PushTransaction` requests;
- `inv` messages generate `AdvertiseBlock` or `AdvertiseTransactions`
requests.
Finally, it refactors the request routing logic for the peer set to
handle advertisement messages, providing three routing methods:
- `route_p2c`, which uses p2c as normal (default);
- `route_inv`, which uses the inventory registry and falls back to p2c
(used for `BlocksByHash` or `TransactionsByHash`);
- `route_all`, which broadcasts a request to all ready peers (used for
`AdvertiseBlock` and `AdvertiseTransactions`).
* network: fix bug in inventory advertisement handling
The RFC https://zebra.zfnd.org/dev/rfcs/0003-inventory-tracking.html described
the use of a `broadcast` channel in place of an `mpsc` channel to get
ring-buffer behavior, keeping a bound on the size of the channel but dropping
old entries when the channel is full.
However, it didn't explicitly describe how this works (the `broadcast` channel
returns a `RecvError::Lagged(u64)` to inform receivers that they lost
messages), so the lag-handling wasn't implemented and I didn't notice in
review.
Instead, the ? operator bubbled the lag error all the way up from
`InventoryRegistry::poll_inventory` through `<PeerSet as Service>::poll_ready`
through various Tower wrappers to users of the peer set. The error propagation
is bad enough, because it caused client errors that shouldn't have happened,
but there's a worse interaction.
The `Service` contract distinguishes between request errors (from
`Service::call`, scoped to the request) and service errors (from
`Service::poll_ready`, scoped to the service). The `Service` contract
specifies that once a service returns an error from `poll_ready`, the service
can be assumed to be failed permanently.
I believe (but haven't tested or carefully worked through the details) that
this caused various tower middleware to report the entire peer set service as
permanently failed due to a transient inventory "error" (more of an indicator),
and I suspect that this is the cause of #1003, where all of the sync
component's requests end up failing because the peer set reported that it
failed permanently. I am able to reproduce #1003 locally before this change
and unable to reproduce it locally after this change, though I have not tested
exhaustively.
* network: add metric for dropped inventory advertisements
Co-authored-by: teor <teor@riseup.net>
Co-authored-by: teor <teor@riseup.net>
The relay flag in the version message is used in conjunction with BIP37 to
receive bloom-filtered transactions. When it is set to false, transactions are
not relayed until a bloom filter is set. Since we don't implement BIP37 (it's
not useful for shielded transactions), this means we'll never receive
transactions.
This is the first in a sequence of changes that change the block:: items
to not include Block as a prefix in their name, in accordance with the
Rust API guidelines.
This extracts the SHA256d code from being split across two modules and puts it
in one module, under serialization.
The code is unchanged except for three deleted tests:
* `sha256d_flush` in `sha256d_writer` (not a meaningful test);
* `transactionhash_debug` (constructs an invalid transaction hash, and the
behavior is tested in the next test);
* `decode_state_debug` (we do not need to test the Debug output of
DecodeState);
* add bytes read and written metrics
* Apply suggestions from code review
Co-authored-by: Jane Lusby <jlusby42@gmail.com>
* store address as string
* Apply suggestions from code review
Co-authored-by: Henry de Valence <hdevalence@hdevalence.ca>
* change addr to label
Co-authored-by: Henry de Valence <hdevalence@hdevalence.ca>
* remove newline
Co-authored-by: Jane Lusby <jlusby42@gmail.com>
Co-authored-by: Henry de Valence <hdevalence@hdevalence.ca>
* checkpoint: reject older of duplicate verification requests.
If we get a duplicate block verification request, we should drop the older one
in favor of the newer one, because the older request is likely to have been
canceled. Previously, this code would accept up to four duplicate verification
requests, then fail all subsequent ones.
* sync: add a timeout layer to block requests.
Note that if this timeout is too short, we'll bring down the peer set in a
retry storm.
* sync: restart syncing on error
Restart the syncing process when an error occurs, rather than ignoring it.
Restarting means we discard all tips and start over with a new block locator,
so we can have another chance to "unstuck" ourselves.
* sync: additional debug info
* sync: handle lookahead limit correctly.
Instead of extracting all the completed task results, the previous code pulled
results out until there were fewer tasks than the lookahead limit, then
stopped. This meant that completed tasks could be left until the limit was
exceeded again. Instead, extract all completed results, and use the number of
pending tasks to decide whether to extend the tip or wait for blocks to finish.
* network: add debug instrumentation to retry policy
* sync: instrument the spawned task
* sync: streamline ObtainTips/ExtendTips logic & tracing
This change does three things:
1. It aligns the implementation of ObtainTips and ExtendTips so that they use
the same deduplication method. This means that when debugging we only have one
deduplication algorithm to focus on.
2. It streamlines the tracing output to not include information already
included in spans. Both obtain_tips and extend_tips have their own spans
attached to the events, so it's not necessary to add Scope: prefixes in
messages.
3. It changes the messages to be focused on reporting the actual
events rather than the interpretation of the events (e.g., "got genesis hash in
response" rather than "peer could not extend tip"). The motivation for this
change is that when debugging, the interpretation of events is already known to
be incorrect, in the sense that the mental model of the code (no bug) does not
match its behavior (has bug), so presenting minimally-interpreted events forces
interpretation relative to the actual code.
* sync: hack to work around zcashd behavior
* sync: localize debug statement in extend_tips
* sync: change algorithm to define tips as pairs of hashes.
This is different enough from the existing description that its comments no
longer apply, so I removed them. A further chunk of work is to change the sync
RFC to document this algorithm.
* sync: reduce block timeout
* state: add resource limits for sled
Closes#888
* sync: add a restart timeout constant
* sync: de-pub constants
* network: move gossiped peer selection logic into address book.
* network: return BoxService from init.
* zebrad: add note on why we truncate thegossiped peer list
Co-authored-by: Jane Lusby <jlusby42@gmail.com>
* Remove unused .rustfmt.toml
Many of these options are never actually loaded by our CI because of a channel
mismatch, where they're not applied on stable but only on nightly (see the logs
from a rustfmt job). This means that we can get different settings when
running `cargo fmt` on the nightly and stable channels, which was causing a CI
failure on this PR. Reverting back to the default rustfmt settings avoids this
problem and keeps us in line with upstream rustfmt. There's no loss to us
since we were using the defaults anyways.
Co-authored-by: Jane Lusby <jlusby42@gmail.com>
Closes#536.
This removes:
- the user-agent (we can add a mechanism to specify extra BIP14 components later, if any users ask us for that feature);
- the EWMA parameters (these were put in the config just to avoid making a choice);
- the peer connection timeout (we can change the default value if anyone ever has a problem with it);
- the peer set request buffer size (setting this too low can make the application deadlock);
The new peer interval is left in.
We can use this network upgrade to implement different consensus rules
and chain context handling for genesis blocks.
Part of the chain state design in #682.
We had a brief discussion on discord and it seemed like we had consensus on the
following versioning policy:
* zebrad: match major version to NU version, so we will start by releasing
zebrad 3.0.0;
* zebra-* libraries: start by matching zebrad's version, then increment major
versions of each library as we need to make breaking changes (potentially
faster than the zebrad version, always respecting semver but making no
guarantees about the longevity of major releases).
This commit sets all of the crate versions to 3.0.0-alpha.0 -- the -alpha.0
marks it as a prerelease not subject to perfect adherence to compatibility
guarantees.
When the connection sees the client_rx channel close it knows it will never get
any more requests, and it should terminate. But instead of terminating, it
errored itself, and the method to error itself tries to pull all the
outstanding client requests from the channel in order to fail them before it
shuts down. This results in reading from a closed channel, causing a panic.
Instead we return cleanly rather than failing (since we know there are no
outstanding requests, as the channel is closed).
This fixes a bug introduced when we added heartbeat support. Recall that we
handle the Bitcoin connection state machine on a per-peer basis. Each
connection has a task created from the `Connection` struct, and a `Client:
tower::Service` "frontend" that passes requests to it via a channel. In the
`Connection` event loop, the connection checks whether the request channel has
been closed, indicating no further requests from the `Client`, in which case it
shuts itself down and cleans up resources. This occurs when all of the senders
have been dropped.
However, this behavior broke when we introduced heartbeat support, because we
spawned an additional task to send heartbeat messages along the request
channel. This meant that instead of having a single sender, dropped by the
`Client`, we have two senders, the `Client` and the "shadow client" task that
generates heartbeat messages. This means that when the `Client` is dropped, we
still have a live sender and the connection is not closed. To fix this, the
`Client` now uses a `oneshot` to shut down its corresponding heartbeat task.
This closes all senders.
Each subsection has to have `serde(default)` to get the behaviour we want
(delete all fields except the ones that have been changed); otherwise, we can
delete only entire sections.
Prior to this change, we required that services that are canceled do not
have a cancel handle in the `cancel_handles` list, based on the
assumption that the handle must have been removed in the process of
canceling this service.
This doesn't holding up though, because it is currently possible for us
to have the same peer connect to us multiple times, the second connect
removes the cancel handle of the original connect and inserts it's own
cancel handle in its place. In this scenario, when the first service is
polled for readiness it will see that it has been canceled and go to
clean itself up, but when it asserts that it doesn't have a cancel
handle it will see the cancel handle of the second connect event, which
uses the same key as the first connect, and fail its debug assertion.
This change removes that debug assert on the assumption that it is okay
for a peer to connect multiple times consecutively, and that the correct
behavior in that case is to just cancel the first connection and
continue as normal.
Prior to this change, the service returned by `zebra_network::init` would spawn background tasks that could silently fail, causing unexpected errors in the zebra_network service.
This change modifies the `PeerSet` that backs `zebra_network::init` to store all of the `JoinHandle`s for each background task it depends on. The `PeerSet` then checks this set of futures to see if any of them have exited with an error or a panic, and if they have it returns the error as part of `poll_ready`.
Co-authored-by: Jane Lusby <jane@zfnd.org>
Prior to this change, the seed subcommand would consistently encounter a panic in one of the background tasks, but would continue running after the panic. This is indicative of two bugs.
First, zebrad was not configured to treat panics as non recoverable and instead defaulted to the tokio defaults, which are to catch panics in tasks and return them via the join handle if available, or to print them if the join handle has been discarded. This is likely a poor fit for zebrad as an application, we do not need to maximize uptime or minimize the extent of an outage should one of our tasks / services start encountering panics. Ignoring a panic increases our risk of observing invalid state, causing all sorts of wild and bad bugs. To deal with this we've switched the default panic behavior from `unwind` to `abort`. This makes panics fail immediately and take down the entire application, regardless of where they occur, which is consistent with our treatment of misbehaving connections.
The second bug is the panic itself. This was triggered by a duplicate entry in the initial_peers set. To fix this we've switched the storage for the peers from a `Vec` to a `HashSet`, which has similar properties but guarantees uniqueness of its keys.
- Add a total peers metric to prevent races between measurements of
ready/unready peers (which can cause the sum to be wrong).
- Add an outbound request counter.
tower-buffer uses tokio's mpsc channels, not the futures-rs mpsc channels.
Unlike futures-rs mpsc channels, which have capacity n+m, where n is the buffer
size and m is the number of senders, tokio channels always have buffer size n.
This means that the buffer size is shared across all peer set handles.
Thanks to @hawkw for sharing details of the Tokio internals!
Previously, we relied on the owner of the handshake future to drive it to
completion. This meant that there were cases where handshakes might never be
completed, just because nothing was actively polling them.
The previous outbound peer connection logic got requests to connect to new
peers and processed them one at a time, making single connection attempts
and retrying if the connection attempt failed. This was quite slow, because
many connections fail, and we have to wait for timeouts. Instead, this logic
connects to new peers concurrently (up to 50 at a time).
Bitcoin does this either with `getblocks` (returns up to 500 following block
hashes) or `getheaders` (returns up to 2000 following block headers, not
just hashes). However, Bitcoin headers are much smaller than Zcash
headers, which contain a giant Equihash solution block, and many Zcash
blocks don't have many transactions in them, so the block header is
often similarly sized to the block itself. Because we're
aiming to have a highly parallel network layer, it seems better to use
`getblocks` to implement `FindBlocks` (which is necessarily sequential)
and parallelize the processing of the block downloads.
This doesn't clean the warnings about unused items in the builder, since
those are unused for a reason (the implementation that should use them
is missing).
PushPeers is more complicated to thread into the rest of our
architecture (we would need to establish a data path connecting our
service handling inbound requests to the network layer's auto-crawler),
and since we crawl the network automatically anyways, we don't actually
need to accept them in order to get updated address information.
The only possible problem with this approach is that zcashd refuses to
answer multiple address requests from the same connection, ostensibly
for fingerprinting prevention (although it's totally happy to give
exactly the same information, as long as you hang up and reconnect
first, lol). It's unclear how this will interact with our design -- on
the one hand, it could mean that we don't get new addr information when
we ask, but on the other hand, we may have enough churn in our
connection pool that this isn't a problem anyways.
Attempting to implement requests for block data revealed a problem with
the previous connection logic. Block data is requested by sending a
`getdata` message with hashes of the requested blocks; the peer responds
with a sequence of `block` messages with the blocks themselves.
However, this wasn't possible to handle with the previous connection
logic, which could only convert a single Bitcoin message into a
Response. Instead, we factor out the message handling logic into a
Handler, which can statefully accumulate arbitrary data into a Response
and signal completion. This is still pretty ugly but it does work.
As a side effect, the HeartbeatNonceMismatch error is removed; because
the Handler now tries to process messages until it comes to a Response,
it just ignores mismatched nonces (and will eventually time out).
The previous Mempool and Transaction requests were removed but could be
re-added in a different form later. Also, the `Get` prefixes are
removed from `Request` to tidy the name.
Closes#158.
As discussed on the issue, this makes it possible to safely serialize
data into hashes, and encourages serializable data to make illegal
states unrepresentable.
These are included in the Block, Transaction objects themselves, so the
previous code ended up trying to deserialize two version fields per
object.
Closes#226.
This replaces the read_list function and makes the code significantly cleaner.
The only downside is that it loses exact preallocation, but this is probably not a big deal.
This means that all sub-modules of `peer` can import everything they need from
the `peer` module itself, without having to be aware of the internal structure
of their sibling modules.
With a 'Transactions' response that gets turned into an 'Inv(Vec<InventoryHash::Tx>)' message.
We don't yet handle a response from our peer for a 'mempool', which will have to be
a more generic 'Inv' type because we might receive transaction hashes we don't know about yet.
Pertains to #26
This does not yet push requests into services that actually respond with transaction
hashes in our node's mempool, which doesn't exist yet.
Pertains to #26
Moved SeedService out of the command closure Command currently spawns
a tokio task to DOS the seed service with `Request::GetPeers` every
second.
Pertains to #54
It's only responsible for doing the handshakes, so it should be named that way,
and then we can have a Connector responsible for actually opening the TCP
connection.
The toml serializer function we are using -- maybe because of to_string_pretty
(?) barfs on structs that mix ordering of simple values and "tables", so just
keep all the Durations to the end.
This splits out the connection handling code into a try_connect closure, which
could be refactored into a Service of its own.
On creation, when we are likely to have very few peers, launch many concurrent
connections to the first few candidates in the initial candidate set, before
continuing to grow the peer set according to demand signals.
The previous implementation failed when timestamps were duplicated between
peers, because there was not a 1-1 relationship between timestamps and peers.
The disconnected_peers() function allows us to prevent duplicate
connections without maintaining shared state between the peerset and the
dial-additional-peers task.
Previously, the TimestampCollector was intended to own the address book
data, so it was intended to be cloneable and hold shared state among all
of its handles. This is now modeled more directly by an
`Arc<Mutex<AddressBook>>`, so the only functionality left in the
`TimestampCollector` is setting up the inital worker, which is better
called `spawn` than `new`.
This also fixes a problem introduced in the previous commit where the
`TimestampCollector` was dropped, causing the worker task to shut down
early.
This allows us to hide the `TimestampCollector` and to expose only the
address book data required by the inbound request service. It also lets
us have a common data structure (the `AddressBook`) for collecting peer
information that can be used to manage information that other peers
report to us.
Doctests can only test public API, so now that the Codec is private, we can't
have a doctest. Since this test was only a code example (no behaviour test),
there's no value in replacing it with a unit test.
This gives API consumers a convenient name, and makes the Rustdoc output
significantly cleaner (because `init` can return a `BoxedZebraService`, not a
`Box<dyn ...ManyTypeConstraints.......>`.
Now that the `PeerConnector` handles both incoming and outgoing
handshakes, determining the next peer address is definitely out of scope
-- it takes a pre-existing tcp connection.
Failure uses a distinct Fail trait rather than the standard library's
Error trait, which causes a lot of interoperability problems with tower
and other Error-using crates. Since failure was created, the standard
library's Error trait was improved, and its conveniences are now
available without the custom Fail trait using `thiserror` (for easy
error derives) and `anyhow` (for a better boxed Error).
* Don't expose submodules of zebra_network::peer.
* PeerSet, PeerDiscover stubs.
Co-authored-by: Deirdre Connolly <deirdre@zfnd.org>
* Initial work on PeerSet.
This is adapted from the MIT-licensed tower-balance implementation.
* Use PeerSet in the connect stub.
This adds a type alias, BoxedStdError, for a boxed std::error::Error
trait object, and uses it in the where bounds for the generic service
code. In the future, we may want to standardize on using
std::error::Error exclusively, but we would then possibly lose out on
backtrace information.
* Fix authorship, license information.
I *thought* I had done a sed pass over the Cargo defaults when doing
repository initialization, but I guess I missed it or something.
Anyways, fixed now.
Add a tower-based peer implementation.
Tower provides middleware for request-response oriented protocols, while Bitcoin/Zcash just send messages which could be interpreted either as requests or responses, depending on context. To bridge this mismatch we define our own internal request/response protocol, and implement a per-peer event loop that scans incoming messages and interprets them either as requests from the remote peer to our node, or as responses to requests we made previously. This is performed by the `PeerService` task, and a corresponding `PeerClient: tower::Service` can send it requests. These tasks are themselves created by a `PeerConnector: tower::Service` which dials a remote peer and performs a handshake.
This field is called `services` in Bitcoin and Zcash, but because we use
that word internally for other purposes, calling it `PeerServices`
disambiguates the meaning to "the services advertised by the peer",
rather than, e.g., a `tower::Service`.
I don't feel super strongly about this change, so I'm happy to drop it,
but it makes the parsing match statements line up nicely and aligns
naming with the naming used in Bitcoin.
This removes the inventory vector structs from `zebra-chain` (as they
are really part of the network protocol) and refactors them into a
single `InventoryHash` type. This corresponds to Bitcoin's "inventory
vector" but with a different, better name (it's not a vector, it's just
a typed hash of some other item).
This provides a significantly cleaner API to consumers, because it
allows using adaptors that convert a TCP stream to a stream of messages,
and potentially allows more efficient message handling.
Because we want to be able to read messages from async sources (like a
tcp socket), we need to have at least async header parsing logic, so
that we can correctly determine how many bytes to await to parse each
message, so it makes sense for the entire message parsing functions
to be async.
Because we perform message serialization into async readers and writers
in the context of sending messages over the network, code using these
functions is more clear with these names.
When we perform a handshake with a remote peer, we need to encode the
version messages with a particular network version before we find out
what the remote peer's version preference is. So in addition to having
a CURRENT_VERSION constant (which represents our preference), we need to
have a MIN_VERSION during the handshake (and later to determine whether
we'll talk to the peer at all).
This adds convenience methods to `ReadZcashExt` that read 4 and 12 byte
fixed size arrays from the `Reader`, making the actual parsing code more
legible.
Closes#10.
* Replace Version MetaAddr by (Services, SocketAddr).
The version handshake message doesn't include last-seen timestamps for
the address fields, unlike other messages, so instead of modeling the
message data with a `MetaAddr` (which includes a timestamp), we should
just use a tuple.
* Simplify try_read_version implementation.
Because we no longer need to construct fake timestamps for the
`MetaAddr` fields, we don't need to use any of the parsed fields while
parsing later fields, and we can neatly wrap up the entire parsing logic
into a single expression.
* fmt
I didn't have the toolchain-specified `rustfmt` because I was mostly
offline and couldn't download it.
This has a test that the serialization implementation round trips
correctly, but is very much a work in progress. Issues with this code
include:
The deserialization logic for message headers is somewhat ugly because
of a lack of a convenient idiom for reading a few bytes at a time into a
fixed-size array. Perhaps this could be fixed with an extension trait,
or avoided altogether by using `u32`s instead of `[u8; 4]`, even when
the latter is the "morally correct type".
Deserialization does an extra allocation, copying the body into a
temporary buffer. This avoids two problems: 1) capping the number of
bytes that can be read by the `Read`er passed into the body parser and
2) allows making two passes over the body data, one to parse it and one
to compute the checksum.
We could avoid making two passes over the body by computing the checksum
simultaneously with the parsing. A convenient way to do this would be to
define a
```
struct ChecksumReader<R: Read> {
inner: R,
digest: Sha256,
}
impl<R: Read> Read for ChecksumReader<R> { /* ... */ }
```
and implement `Read` for `ChecksumReader` to forward reads from the
inner reader, copying data into the digest object as it does so. (It
could also have a maximum length to enforce that we do not read past the
nominal end of the body).
A similar `ChecksumWriter` could be used during serialization, although
because the checksum is at the beginning rather than the end of the
message it does not help avoid an allocation there. It could also be
generic over a `Digest` implementation, although because we need a
truncated double-SHA256 we would have to write a custom `Digest`
implementation, so this is probably not worthwhile unless we have other
checksum types.
Finally, this does very minimal testing -- just round-trip serialization
on a single message. It would be good to build in support for
property-based testing, probably using `proptest`; if we could define
generation and shrinking strategies for every component type of every
message, we could do strong randomized testing of the serialization.
The core serialization logic is now in zebra-chain and consists of two
pairs of traits:
These are analogues of the Serde `Serialize` and `Deserialize` traits,
but explicitly intended for consensus-critical serialization formats.
Thus some struct `Foo` may have derived `Serialize` and `Deserialize`
implementations for (internal) use with Serde, and explicitly-written
`ZcashSerialize` and `ZcashDeserialize` implementations for use in
consensus-critical contexts. The consensus-critical implementations
provide `zcash`-prefixed `zcash_serialize` and `zcash_deserialize`
methods to make it clear in client contexts that the serialization is
consensus-critical.
These are utility traits, analogous to the `ReadBytesExt` and
`WriteBytesExt` traits provided by `byteorder`. A generic
implementation is provided for any `io::Read` or `io::Write`, so that
bringing the traits into scope adds additional Zcash-specific traits to
generic readers and writers -- for instance, writing a `u64` in the
Bitcoin "CompactSize" format.
Currently these just have write_compactsize and read_compactsize methods which
allow reading and writing u64s to any `Read` or `Write` implementation using
the Bitcoin "CompactSize" variable integer encoding.
These methods read and write u64s rather than defining a new `CompactSize`
type, because the `CompactSize` is just an encoding detail, not a different
type with any distinct meaning.
The `NetworkAddress` type was a `(Services, SocketAddr)` pair as used in the
`version` handshake message, described as the `net_addr` struct in the Bitcoin
wiki protocol documentation. However, all of the other uses of the `net_addr`
struct are a `(Timestamp, Services, SocketAddr)` pair (where the timestamp is
the last-seen time of the peer), and the timestamp is omitted only during the
`version` messages, which are used only during the handshake, so it seems
better to include the timestamp field and omit it during serialization of
`version` packets.