edenapi: Hide edenapi wire types from externally visible API

Summary:
Introduce separate wire types to allow protocol evolution and client API changes to happen independently.

* Duplicate `*Request`, `*Entry`, `Key`, `Parents`, `RepoPathBuf`, `HgId`, and `revisionstore_types::Metadata` types into the `wire` module. The versions in the `wire` module are required to have proper `serde` annotations, `Serialize` / `Deserialize` implementations, etc. These have been removed from the original structs.
* Introduce infallible conversions from "API types" to "wire types" with the `ToWire` trait and fallible conversions from "wire types" to "API types" with the `ToApi`. API -> wire conversions should never fail in a binary that builds succesfully, but wire -> API conversions can fail in the case that the server and client are using different versions of the library. This will cause, for instance, a newly-introduced enum variant used by the client to be deserialized into the catch-all `Unknown` variant on the server, which won't generally have a corresponding representation in the API type.
* Cleanup: remove `*Response` types, which are no longer used anywhere.
* Introduce a `map` method on `Fetch` struct which allows a fallible conversion function to be used to convert a `Fetch<T>` to a `Fetch<U>`. This function is used in the edenapi client implementation to convert from wire types to API types.
* Modify `edenapi_server` to convert from API types to wire types.
* Modify `edenapi_cli` to convert back to wire types before serializing responses to disk.
* Modify `make_req` to use `ToWire` for converting API structs from the `json` module to wire structs.
* Modify `read_res` to use `ToApi` to convert deserialized wire types to API types with the necessary methods for investigating the contents (`.data()`, primarily). It will print an error message to stderr if it encounters a wire type which cannot be converted into the corresponding API type.
* Add some documentation about protocol conventions to the root of the `wire` module.

Reviewed By: kulshrax

Differential Revision: D23224705

fbshipit-source-id: 88f8addc403f3a8da3cde2aeee765899a826446d
This commit is contained in:
Meyer Jacobs 2020-09-23 17:25:18 -07:00 committed by Facebook GitHub Bot
parent a745a145b1
commit 75105421ce
23 changed files with 1049 additions and 176 deletions

View File

@ -6,12 +6,15 @@
*/
use anyhow::Error;
use futures::{Stream, TryStreamExt};
use futures::{Stream, StreamExt, TryStreamExt};
use gotham::state::{FromState, State};
use gotham_derive::{StateData, StaticResponseExtender};
use serde::Deserialize;
use edenapi_types::{CompleteTreeRequest, TreeEntry};
use edenapi_types::{
wire::{ToApi, ToWire, WireCompleteTreeRequest},
CompleteTreeRequest, TreeEntry,
};
use gotham_ext::{error::HttpError, response::TryIntoResponse};
use mercurial_types::{HgManifestId, HgNodeHash};
use mononoke_api::{
@ -42,9 +45,18 @@ pub async fn complete_trees(state: &mut State) -> Result<impl TryIntoResponse, H
let sctx = ServerContext::borrow_from(state);
let repo = get_repo(&sctx, &rctx, &params.repo).await?;
let request = parse_cbor_request(state).await?;
let request: WireCompleteTreeRequest = parse_cbor_request(state).await?;
let request: CompleteTreeRequest = match request.to_api() {
Ok(r) => r,
Err(e) => {
return Err(HttpError::e400(e));
}
};
Ok(cbor_stream(rctx, fetch_trees_under_path(&repo, request)?))
Ok(cbor_stream(
rctx,
fetch_trees_under_path(&repo, request)?.map(|r| r.map(|e| e.to_wire())),
))
}
/// Fetch the complete tree under the specified path.

View File

@ -11,7 +11,10 @@ use gotham::state::{FromState, State};
use gotham_derive::{StateData, StaticResponseExtender};
use serde::Deserialize;
use edenapi_types::{FileEntry, FileRequest};
use edenapi_types::{
wire::{ToApi, ToWire, WireFileRequest},
FileEntry, FileRequest,
};
use gotham_ext::{error::HttpError, response::TryIntoResponse};
use mercurial_types::{HgFileNodeId, HgNodeHash};
use mononoke_api::hg::{HgDataContext, HgDataId, HgRepoContext};
@ -42,9 +45,18 @@ pub async fn files(state: &mut State) -> Result<impl TryIntoResponse, HttpError>
let sctx = ServerContext::borrow_from(state);
let repo = get_repo(&sctx, &rctx, &params.repo).await?;
let request = parse_cbor_request(state).await?;
let request: WireFileRequest = parse_cbor_request(state).await?;
let request: FileRequest = match request.to_api() {
Ok(r) => r,
Err(e) => {
return Err(HttpError::e400(e));
}
};
Ok(cbor_stream(rctx, fetch_all_files(repo, request)))
Ok(cbor_stream(
rctx,
fetch_all_files(repo, request).map(|r| r.map(|v| v.to_wire())),
))
}
/// Fetch files for all of the requested keys concurrently.

View File

@ -11,7 +11,10 @@ use gotham::state::{FromState, State};
use gotham_derive::{StateData, StaticResponseExtender};
use serde::Deserialize;
use edenapi_types::{TreeEntry, TreeRequest};
use edenapi_types::{
wire::{ToApi, ToWire, WireTreeRequest},
TreeEntry, TreeRequest,
};
use gotham_ext::{error::HttpError, response::TryIntoResponse};
use mercurial_types::{HgManifestId, HgNodeHash};
use mononoke_api::hg::{HgDataContext, HgDataId, HgRepoContext};
@ -42,9 +45,18 @@ pub async fn trees(state: &mut State) -> Result<impl TryIntoResponse, HttpError>
let sctx = ServerContext::borrow_from(state);
let repo = get_repo(&sctx, &rctx, &params.repo).await?;
let request = parse_cbor_request(state).await?;
let request: WireTreeRequest = parse_cbor_request(state).await?;
let request: TreeRequest = match request.to_api() {
Ok(r) => r,
Err(e) => {
return Err(HttpError::e400(e));
}
};
Ok(cbor_stream(rctx, fetch_all_trees(repo, request)))
Ok(cbor_stream(
rctx,
fetch_all_trees(repo, request).map(|r| r.map(|v| v.to_wire())),
))
}
/// Fetch trees for all of the requested keys concurrently.

View File

@ -92,16 +92,16 @@ Create and send complete tree request.
> }
> EOF
Reading from stdin
Generated request: CompleteTreeRequest {
rootdir: RepoPathBuf(
Generated request: WireCompleteTreeRequest {
rootdir: WireRepoPathBuf(
"",
),
mfnodes: [
HgId("3d866afaa8cdb847e3800fef742c1fe9e741f75f"),
HgId("8cad2f4cf4dc3d149356ed44a973fd3f6284deb6"),
WireHgId("3d866afaa8cdb847e3800fef742c1fe9e741f75f"),
WireHgId("8cad2f4cf4dc3d149356ed44a973fd3f6284deb6"),
],
basemfnodes: [
HgId("63e28e06687f0750555703a5993d72665ed21467"),
WireHgId("63e28e06687f0750555703a5993d72665ed21467"),
],
depth: Some(
2,

View File

@ -42,19 +42,19 @@ Create and send file request.
> }
> EOF
Reading from stdin
Generated request: FileRequest {
Generated request: WireFileRequest {
keys: [
Key {
path: RepoPathBuf(
WireKey {
path: WireRepoPathBuf(
"test.txt",
),
hgid: HgId("186cafa3319c24956783383dc44c5cbc68c5a0ca"),
hgid: WireHgId("186cafa3319c24956783383dc44c5cbc68c5a0ca"),
},
Key {
path: RepoPathBuf(
WireKey {
path: WireRepoPathBuf(
"copy.txt",
),
hgid: HgId("17b8d4e3bafd4ec4812ad7c930aace9bf07ab033"),
hgid: WireHgId("17b8d4e3bafd4ec4812ad7c930aace9bf07ab033"),
},
],
}

View File

@ -42,19 +42,19 @@ Create and send tree request.
> }
> EOF
Reading from stdin
Generated request: TreeRequest {
Generated request: WireTreeRequest {
keys: [
Key {
path: RepoPathBuf(
WireKey {
path: WireRepoPathBuf(
"",
),
hgid: HgId("15024c4dc4a27b572d623db342ae6a08d7f7adec"),
hgid: WireHgId("15024c4dc4a27b572d623db342ae6a08d7f7adec"),
},
Key {
path: RepoPathBuf(
WireKey {
path: WireRepoPathBuf(
"",
),
hgid: HgId("c8743b14e0789cc546125213c18a18d813862db5"),
hgid: WireHgId("c8743b14e0789cc546125213c18a18d813862db5"),
},
],
}

View File

@ -21,8 +21,8 @@ use tokio::prelude::*;
use configparser::config::{ConfigSet, Options};
use edenapi::{Builder, Client, EdenApi, Entries, Fetch, Progress, ProgressCallback};
use edenapi_types::{
json::FromJson, CommitRevlogDataRequest, CompleteTreeRequest, FileRequest, HistoryRequest,
TreeRequest,
json::FromJson, wire::ToWire, CommitRevlogDataRequest, CompleteTreeRequest, FileRequest,
HistoryRequest, TreeRequest,
};
const DEFAULT_CONFIG_FILE: &str = ".hgrc.edenapi";
@ -127,7 +127,7 @@ async fn cmd_history(args: Args) -> Result<()> {
let res = client
.history(repo.clone(), req.keys, req.length, Some(cb))
.await?;
handle_response(res, bar).await?;
handle_response_raw(res, bar).await?;
}
Ok(())
@ -196,7 +196,7 @@ async fn cmd_commit_revlog_data(args: Args) -> Result<()> {
let res = client
.commit_revlog_data(repo.clone(), req.hgids, Some(cb))
.await?;
handle_response(res, bar).await?;
handle_response_raw(res, bar).await?;
}
Ok(())
@ -205,7 +205,7 @@ async fn cmd_commit_revlog_data(args: Args) -> Result<()> {
/// Handle the incoming deserialized response by reserializing it
/// and dumping it to stdout (only if stdout isn't a TTY, to avoid
/// messing up the user's terminal).
async fn handle_response<T: Serialize>(res: Fetch<T>, bar: ProgressBar) -> Result<()> {
async fn handle_response<T: ToWire>(res: Fetch<T>, bar: ProgressBar) -> Result<()> {
let buf = serialize_and_concat(res.entries).await?;
let stats = res.stats.await?;
@ -224,6 +224,26 @@ async fn handle_response<T: Serialize>(res: Fetch<T>, bar: ProgressBar) -> Resul
Ok(())
}
// TODO(meyer): Remove when all types have wire type
async fn handle_response_raw<T: Serialize>(res: Fetch<T>, bar: ProgressBar) -> Result<()> {
let buf = serialize_and_concat_raw(res.entries).await?;
let stats = res.stats.await?;
bar.finish_at_current_pos();
log::info!("{}", &stats);
log::trace!("Response metadata: {:#?}", &res.meta);
if atty::is(atty::Stream::Stdout) {
log::warn!("Not writing output because stdout is a TTY");
} else {
log::info!("Writing output to stdout");
io::stdout().write_all(&buf).await?;
}
Ok(())
}
/// CBOR serialize and concatenate all items in the incoming stream.
///
/// Normally, this wouldn't be a good idea since the EdenAPI client just
@ -231,7 +251,16 @@ async fn handle_response<T: Serialize>(res: Fetch<T>, bar: ProgressBar) -> Resul
/// However, in this case we're explicitly trying to exercise the public API
/// of the client, including deserialization. In practice, most users will
/// never want the raw (CBOR-encoded) entries.
async fn serialize_and_concat<T: Serialize>(entries: Entries<T>) -> Result<Vec<u8>> {
async fn serialize_and_concat<T: ToWire>(entries: Entries<T>) -> Result<Vec<u8>> {
entries
.err_into()
.and_then(|entry| async move { Ok(serde_cbor::to_vec(&entry.to_wire())?) })
.try_concat()
.await
}
// TODO: Remove when all types have wire type
async fn serialize_and_concat_raw<T: Serialize>(entries: Entries<T>) -> Result<Vec<u8>> {
entries
.err_into()
.and_then(|entry| async move { Ok(serde_cbor::to_vec(&entry)?) })

View File

@ -15,6 +15,7 @@ use serde::{de::DeserializeOwned, Serialize};
use url::Url;
use edenapi_types::{
wire::{ToApi, ToWire, WireFileEntry, WireTreeEntry},
CommitRevlogData, CommitRevlogDataRequest, CompleteTreeRequest, FileEntry, FileRequest,
HistoryEntry, HistoryRequest, HistoryResponseChunk, TreeEntry, TreeRequest,
};
@ -107,7 +108,7 @@ impl Client {
.collect()
}
/// Fetch data from the server.
/// Fetch data from the server without Wire to Api conversion.
///
/// Concurrently performs all of the given HTTP requests, each of
/// which must result in streaming response of CBOR-encoded values
@ -115,7 +116,7 @@ impl Client {
/// the order the responses arrive. The response streams will be
/// combined into a single stream, in which the returned entries
/// from different HTTP responses may be arbitrarily interleaved.
async fn fetch<T: DeserializeOwned + Send + 'static>(
async fn fetch_raw<T: DeserializeOwned + Send + 'static>(
&self,
requests: Vec<Request>,
progress: Option<ProgressCallback>,
@ -145,6 +146,55 @@ impl Client {
stats,
})
}
/// Fetch data from the server.
///
/// Concurrently performs all of the given HTTP requests, each of
/// which must result in streaming response of CBOR-encoded values
/// of type `T`. The metadata of each response will be returned in
/// the order the responses arrive. The response streams will be
/// combined into a single stream, in which the returned entries
/// from different HTTP responses may be arbitrarily interleaved.
async fn fetch<T>(
&self,
requests: Vec<Request>,
progress: Option<ProgressCallback>,
) -> Result<Fetch<<T as ToApi>::Api>, EdenApiError>
where
T: ToApi + Send + DeserializeOwned + 'static,
<T as ToApi>::Api: Send + 'static,
{
let progress = progress.unwrap_or_else(|| Box::new(|_| ()));
let requests = requests.into_iter().collect::<Vec<_>>();
let n_requests = requests.len();
let (mut responses, stats) = self.client.send_async_with_progress(requests, progress)?;
let mut meta = Vec::with_capacity(n_requests);
let mut streams = Vec::with_capacity(n_requests);
while let Some(res) = responses.try_next().await? {
meta.push(ResponseMeta::from(&res));
let entries = res
.into_cbor_stream::<T>()
.map(|r| {
r.map_err(|e| EdenApiError::from(e))
.and_then(|v| v.to_api().map_err(|e| EdenApiError::from(e.into())))
})
.boxed();
streams.push(entries);
}
let entries = stream::select_all(streams).boxed();
let stats = stats.err_into().boxed();
Ok(Fetch {
meta,
entries,
stats,
})
}
}
#[async_trait]
@ -173,11 +223,11 @@ impl EdenApi for Client {
}
let url = self.url(paths::FILES, Some(&repo))?;
let requests = self.prepare(&url, keys, self.config.max_files, |keys| FileRequest {
keys,
let requests = self.prepare(&url, keys, self.config.max_files, |keys| {
FileRequest { keys }.to_wire()
})?;
self.fetch::<FileEntry>(requests, progress).await
Ok(self.fetch::<WireFileEntry>(requests, progress).await?)
}
async fn history(
@ -203,7 +253,7 @@ impl EdenApi for Client {
entries,
stats,
} = self
.fetch::<HistoryResponseChunk>(requests, progress)
.fetch_raw::<HistoryResponseChunk>(requests, progress)
.await?;
// Convert received `HistoryResponseChunk`s into `HistoryEntry`s.
@ -232,11 +282,11 @@ impl EdenApi for Client {
}
let url = self.url(paths::TREES, Some(&repo))?;
let requests = self.prepare(&url, keys, self.config.max_trees, |keys| TreeRequest {
keys,
let requests = self.prepare(&url, keys, self.config.max_trees, |keys| {
TreeRequest { keys }.to_wire()
})?;
self.fetch::<TreeEntry>(requests, progress).await
Ok(self.fetch::<WireTreeEntry>(requests, progress).await?)
}
async fn complete_trees(
@ -260,14 +310,15 @@ impl EdenApi for Client {
mfnodes,
basemfnodes,
depth,
};
}
.to_wire();
let req = self
.configure(Request::post(url))?
.cbor(&tree_req)
.map_err(EdenApiError::RequestSerializationFailed)?;
self.fetch::<TreeEntry>(vec![req], progress).await
Ok(self.fetch::<WireTreeEntry>(vec![req], progress).await?)
}
async fn commit_revlog_data(
@ -286,7 +337,8 @@ impl EdenApi for Client {
.cbor(&commit_revlog_data_req)
.map_err(EdenApiError::RequestSerializationFailed)?;
self.fetch::<CommitRevlogData>(vec![req], progress).await
self.fetch_raw::<CommitRevlogData>(vec![req], progress)
.await
}
}

View File

@ -7,6 +7,7 @@
use thiserror::Error;
use edenapi_types::wire::WireToApiConversionError;
use http_client::{CertOrKeyMissing, HttpClientError};
#[derive(Debug, Error)]
@ -24,6 +25,8 @@ pub enum EdenApiError {
#[error(transparent)]
InvalidUrl(#[from] url::ParseError),
#[error(transparent)]
WireToApiConversionFailed(#[from] WireToApiConversionError),
#[error(transparent)]
Other(#[from] anyhow::Error),
}

View File

@ -25,8 +25,8 @@ use serde_json::Value;
use structopt::StructOpt;
use edenapi_types::{
json::FromJson, CommitLocationToHashRequest, CommitRevlogDataRequest, CompleteTreeRequest,
FileRequest, HistoryRequest, TreeRequest,
json::FromJson, wire::ToWire, CommitLocationToHashRequest, CommitRevlogDataRequest,
CompleteTreeRequest, FileRequest, HistoryRequest, TreeRequest,
};
#[derive(Debug, StructOpt)]
@ -52,14 +52,23 @@ fn main() -> Result<()> {
match Command::from_args() {
Command::File(args) => make_req::<FileRequest>(args),
Command::Tree(args) => make_req::<TreeRequest>(args),
Command::History(args) => make_req::<HistoryRequest>(args),
Command::History(args) => make_req_wire::<HistoryRequest>(args),
Command::CompleteTree(args) => make_req::<CompleteTreeRequest>(args),
Command::CommitRevlogData(args) => make_req::<CommitRevlogDataRequest>(args),
Command::CommitLocationToHash(args) => make_req::<CommitLocationToHashRequest>(args),
Command::CommitRevlogData(args) => make_req_wire::<CommitRevlogDataRequest>(args),
Command::CommitLocationToHash(args) => make_req_wire::<CommitLocationToHashRequest>(args),
}
}
fn make_req<R: FromJson + Serialize + Debug>(args: Args) -> Result<()> {
fn make_req<R: FromJson + ToWire>(args: Args) -> Result<()> {
let json = read_input(args.input)?;
let req = R::from_json(&json)?.to_wire();
let bytes = serde_cbor::to_vec(&req)?;
eprintln!("Generated request: {:#?}", &req);
write_output(args.output, &bytes)
}
// TODO: Remove after all requests standarize to match FileRequest, TreeRequest, CompleteTreeRequest
fn make_req_wire<R: FromJson + Serialize + Debug>(args: Args) -> Result<()> {
let json = read_input(args.input)?;
let req = R::from_json(&json)?;
let bytes = serde_cbor::to_vec(&req)?;

View File

@ -24,8 +24,9 @@ use sha1::{Digest, Sha1};
use structopt::StructOpt;
use edenapi_types::{
CommitLocationToHash, CommitRevlogData, FileEntry, FileError, HistoryResponseChunk, TreeEntry,
TreeError, WireHistoryEntry,
wire::{ToApi, WireFileEntry, WireTreeEntry},
CommitLocationToHash, CommitRevlogData, FileError, HistoryResponseChunk, TreeError,
WireHistoryEntry,
};
use types::{HgId, Key, Parents, RepoPathBuf};
@ -198,8 +199,8 @@ fn cmd_file(args: FileArgs) -> Result<()> {
}
fn cmd_tree_ls(args: DataLsArgs) -> Result<()> {
let entries: Vec<TreeEntry> = read_input(args.input, args.limit)?;
for entry in entries {
let entries: Vec<WireTreeEntry> = read_input(args.input, args.limit)?;
for entry in entries.into_iter().filter_map(to_api) {
println!("{}", entry.key());
}
Ok(())
@ -210,9 +211,10 @@ fn cmd_tree_cat(args: DataCatArgs) -> Result<()> {
let hgid = args.hgid.parse()?;
let key = Key::new(path, hgid);
let entries: Vec<TreeEntry> = read_input(args.input, args.limit)?;
let entries: Vec<WireTreeEntry> = read_input(args.input, args.limit)?;
let entry = entries
.into_iter()
.filter_map(to_api)
.find(|entry| entry.key() == &key)
.ok_or_else(|| anyhow!("Key not found"))?;
@ -220,8 +222,8 @@ fn cmd_tree_cat(args: DataCatArgs) -> Result<()> {
}
fn cmd_tree_check(args: DataCheckArgs) -> Result<()> {
let entries: Vec<TreeEntry> = read_input(args.input, args.limit)?;
for entry in entries {
let entries: Vec<WireTreeEntry> = read_input(args.input, args.limit)?;
for entry in entries.into_iter().filter_map(to_api) {
match entry.data() {
Ok(_) => {}
Err(TreeError::MaybeHybridManifest(e)) => {
@ -236,8 +238,8 @@ fn cmd_tree_check(args: DataCheckArgs) -> Result<()> {
}
fn cmd_file_ls(args: DataLsArgs) -> Result<()> {
let entries: Vec<FileEntry> = read_input(args.input, args.limit)?;
for entry in entries {
let entries: Vec<WireFileEntry> = read_input(args.input, args.limit)?;
for entry in entries.into_iter().filter_map(to_api) {
println!("{}", entry.key());
}
Ok(())
@ -248,9 +250,10 @@ fn cmd_file_cat(args: DataCatArgs) -> Result<()> {
let hgid = args.hgid.parse()?;
let key = Key::new(path, hgid);
let entries: Vec<FileEntry> = read_input(args.input, args.limit)?;
let entries: Vec<WireFileEntry> = read_input(args.input, args.limit)?;
let entry = entries
.into_iter()
.filter_map(to_api)
.find(|entry| entry.key() == &key)
.ok_or_else(|| anyhow!("Key not found"))?;
@ -258,8 +261,8 @@ fn cmd_file_cat(args: DataCatArgs) -> Result<()> {
}
fn cmd_file_check(args: DataCheckArgs) -> Result<()> {
let entries: Vec<FileEntry> = read_input(args.input, args.limit)?;
for entry in entries {
let entries: Vec<WireFileEntry> = read_input(args.input, args.limit)?;
for entry in entries.into_iter().filter_map(to_api) {
match entry.data() {
Ok(_) => {}
Err(FileError::Redacted(..)) => {
@ -426,6 +429,16 @@ fn read_input<T: DeserializeOwned>(path: Option<PathBuf>, limit: Option<usize>)
})
}
fn to_api<T: ToApi>(entry: T) -> Option<T::Api> {
match entry.to_api() {
Ok(api) => Some(api),
Err(_) => {
eprintln!("Failed to convert entry to API type");
None
}
}
}
fn write_output(path: Option<PathBuf>, content: &[u8]) -> Result<()> {
match path {
Some(path) => {

View File

@ -16,6 +16,7 @@ anyhow = "1.0"
bytes = { version = "0.5", features = ["serde"] }
quickcheck = "0.9"
serde = { version = "1.0", features = ["derive", "rc"] }
serde_cbor = "0.11"
serde_derive = "1.0"
serde_json = "1.0"
thiserror = "1.0"

View File

@ -9,8 +9,6 @@ use serde_derive::{Deserialize, Serialize};
use types::{hgid::HgId, path::RepoPathBuf};
use crate::is_default;
/// Struct reprenting the arguments to a "gettreepack" operation, which
/// is used by Mercurial to prefetch treemanifests. This struct is intended
/// to provide a way to support requests compatible with Mercurial's existing
@ -21,18 +19,11 @@ use crate::is_default;
/// containing the keys of the desired tree nodes.
///
/// In all cases, trees will be returned in a `TreeResponse`.
#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
#[derive(Clone, Debug, Eq, Deserialize, Serialize, PartialEq)]
pub struct CompleteTreeRequest {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub rootdir: RepoPathBuf,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
pub mfnodes: Vec<HgId>,
#[serde(rename = "2", default, skip_serializing_if = "is_default")]
pub basemfnodes: Vec<HgId>,
#[serde(rename = "3", default, skip_serializing_if = "is_default")]
pub depth: Option<usize>,
}

View File

@ -5,16 +5,17 @@
* GNU General Public License version 2.
*/
use std::iter::FromIterator;
use bytes::Bytes;
use serde_derive::{Deserialize, Serialize};
use thiserror::Error;
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
use revisionstore_types::Metadata;
use types::{hgid::HgId, key::Key, parents::Parents};
use crate::{is_default, InvalidHgId};
use crate::InvalidHgId;
/// Tombstone string that replaces the content of redacted files.
/// TODO(T48685378): Handle redacted content in a less hacky way.
@ -49,19 +50,12 @@ impl FileError {
/// Structure representing source control file content on the wire.
/// Includes the information required to add the data to a mutable store,
/// along with the parents for hash validation.
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
#[derive(Clone, Debug, Default, Deserialize, Serialize, Eq, PartialEq)]
pub struct FileEntry {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
key: Key,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
data: Bytes,
#[serde(rename = "2", default, skip_serializing_if = "is_default")]
parents: Parents,
#[serde(rename = "3", default, skip_serializing_if = "is_default")]
metadata: Metadata,
pub key: Key,
pub data: Bytes,
pub parents: Parents,
pub metadata: Metadata,
}
impl FileEntry {
@ -121,44 +115,24 @@ impl FileEntry {
}
}
#[derive(Clone, Default, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct FileRequest {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub keys: Vec<Key>,
}
#[derive(Debug, Default, Serialize, Deserialize)]
pub struct FileResponse {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub entries: Vec<FileEntry>,
}
impl FileResponse {
pub fn new(entries: impl IntoIterator<Item = FileEntry>) -> Self {
Self::from_iter(entries)
}
}
impl FromIterator<FileEntry> for FileResponse {
fn from_iter<I: IntoIterator<Item = FileEntry>>(entries: I) -> Self {
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for FileEntry {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
let bytes: Vec<u8> = Arbitrary::arbitrary(g);
Self {
entries: entries.into_iter().collect(),
key: Arbitrary::arbitrary(g),
data: Bytes::from(bytes),
parents: Arbitrary::arbitrary(g),
metadata: Arbitrary::arbitrary(g),
}
}
}
impl IntoIterator for FileResponse {
type Item = FileEntry;
type IntoIter = std::vec::IntoIter<FileEntry>;
fn into_iter(self) -> Self::IntoIter {
self.entries.into_iter()
}
#[derive(Clone, Default, Debug, Deserialize, Serialize, Eq, PartialEq)]
pub struct FileRequest {
pub keys: Vec<Key>,
}
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for FileRequest {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {

View File

@ -31,17 +31,18 @@ pub mod file;
pub mod history;
pub mod json;
pub mod tree;
pub mod wire;
pub use crate::commit::{
CommitLocation, CommitLocationToHash, CommitLocationToHashRequest, CommitRevlogData,
CommitRevlogDataRequest,
};
pub use crate::complete_tree::CompleteTreeRequest;
pub use crate::file::{FileEntry, FileError, FileRequest, FileResponse};
pub use crate::file::{FileEntry, FileError, FileRequest};
pub use crate::history::{
HistoryEntry, HistoryRequest, HistoryResponse, HistoryResponseChunk, WireHistoryEntry,
};
pub use crate::tree::{TreeEntry, TreeError, TreeRequest, TreeResponse};
pub use crate::tree::{TreeEntry, TreeError, TreeRequest};
use thiserror::Error;
@ -56,7 +57,3 @@ pub struct InvalidHgId {
data: Bytes,
parents: Parents,
}
fn is_default<T: Default + PartialEq>(v: &T) -> bool {
v == &T::default()
}

View File

@ -5,16 +5,17 @@
* GNU General Public License version 2.
*/
use std::iter::FromIterator;
use bytes::Bytes;
use serde_derive::{Deserialize, Serialize};
use thiserror::Error;
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
use revisionstore_types::Metadata;
use types::{hgid::HgId, key::Key, parents::Parents};
use crate::{is_default, InvalidHgId};
use crate::InvalidHgId;
#[derive(Debug, Error)]
pub enum TreeError {
@ -44,19 +45,12 @@ impl TreeError {
/// Structure representing source control tree entry on the wire.
/// Includes the information required to add the data to a mutable store,
/// along with the parents for hash validation.
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
#[derive(Clone, Debug, Default, Deserialize, Serialize, Eq, PartialEq)]
pub struct TreeEntry {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
key: Key,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
data: Bytes,
#[serde(rename = "2", default, skip_serializing_if = "is_default")]
parents: Parents,
#[serde(rename = "3", default, skip_serializing_if = "is_default")]
metadata: Metadata,
pub key: Key,
pub data: Bytes,
pub parents: Parents,
pub metadata: Metadata,
}
impl TreeEntry {
@ -115,44 +109,24 @@ impl TreeEntry {
}
}
#[derive(Clone, Debug, Default, Serialize, Deserialize, Eq, PartialEq)]
pub struct TreeRequest {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub keys: Vec<Key>,
}
#[derive(Debug, Default, Serialize, Deserialize)]
pub struct TreeResponse {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub entries: Vec<TreeEntry>,
}
impl TreeResponse {
pub fn new(entries: impl IntoIterator<Item = TreeEntry>) -> Self {
Self::from_iter(entries)
}
}
impl FromIterator<TreeEntry> for TreeResponse {
fn from_iter<I: IntoIterator<Item = TreeEntry>>(entries: I) -> Self {
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for TreeEntry {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
let bytes: Vec<u8> = Arbitrary::arbitrary(g);
Self {
entries: entries.into_iter().collect(),
key: Arbitrary::arbitrary(g),
data: Bytes::from(bytes),
parents: Arbitrary::arbitrary(g),
metadata: Arbitrary::arbitrary(g),
}
}
}
impl IntoIterator for TreeResponse {
type Item = TreeEntry;
type IntoIter = std::vec::IntoIter<TreeEntry>;
fn into_iter(self) -> Self::IntoIter {
self.entries.into_iter()
}
#[derive(Clone, Debug, Default, Deserialize, Serialize, Eq, PartialEq)]
pub struct TreeRequest {
pub keys: Vec<Key>,
}
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for TreeRequest {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {

View File

@ -0,0 +1,99 @@
/*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This software may be used and distributed according to the terms of the
* GNU General Public License version 2.
*/
use serde_derive::{Deserialize, Serialize};
use crate::{
wire::{is_default, ToApi, ToWire, WireHgId, WireRepoPathBuf, WireToApiConversionError},
CompleteTreeRequest,
};
/// Struct reprenting the arguments to a "gettreepack" operation, which
/// is used by Mercurial to prefetch treemanifests. This struct is intended
/// to provide a way to support requests compatible with Mercurial's existing
/// gettreepack wire protocol command.
///
/// In the future, we'd like to migrate away from requesting trees in this way.
/// In general, trees can be requested from the API server using a `TreeRequest`
/// containing the keys of the desired tree nodes.
///
/// In all cases, trees will be returned in a `TreeResponse`.
#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct WireCompleteTreeRequest {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub rootdir: WireRepoPathBuf,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
pub mfnodes: Vec<WireHgId>,
#[serde(rename = "2", default, skip_serializing_if = "is_default")]
pub basemfnodes: Vec<WireHgId>,
#[serde(rename = "3", default, skip_serializing_if = "is_default")]
pub depth: Option<usize>,
}
impl ToWire for CompleteTreeRequest {
type Wire = WireCompleteTreeRequest;
fn to_wire(self) -> Self::Wire {
WireCompleteTreeRequest {
rootdir: self.rootdir.to_wire(),
mfnodes: self.mfnodes.to_wire(),
basemfnodes: self.basemfnodes.to_wire(),
depth: self.depth,
}
}
}
impl ToApi for WireCompleteTreeRequest {
type Api = CompleteTreeRequest;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(CompleteTreeRequest {
rootdir: self.rootdir.to_api()?,
mfnodes: self.mfnodes.to_api()?,
basemfnodes: self.basemfnodes.to_api()?,
depth: self.depth,
})
}
}
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireCompleteTreeRequest {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Self {
rootdir: Arbitrary::arbitrary(g),
mfnodes: Arbitrary::arbitrary(g),
basemfnodes: Arbitrary::arbitrary(g),
depth: Arbitrary::arbitrary(g),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::wire::tests::{check_serialize_roundtrip, check_wire_roundtrip};
use quickcheck::quickcheck;
quickcheck! {
fn test_request_roundtrip_serialize(v: WireCompleteTreeRequest) -> bool {
check_serialize_roundtrip(v)
}
fn test_request_roundtrip_wire(v: CompleteTreeRequest) -> bool {
check_wire_roundtrip(v)
}
}
}

View File

@ -0,0 +1,137 @@
/*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This software may be used and distributed according to the terms of the
* GNU General Public License version 2.
*/
use bytes::Bytes;
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
use serde_derive::{Deserialize, Serialize};
use crate::{
file::{FileEntry, FileRequest},
wire::{
is_default, ToApi, ToWire, WireKey, WireParents, WireRevisionstoreMetadata,
WireToApiConversionError,
},
};
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
pub struct WireFileEntry {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
key: WireKey,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
data: Bytes,
#[serde(rename = "2", default, skip_serializing_if = "is_default")]
parents: WireParents,
#[serde(rename = "3", default, skip_serializing_if = "is_default")]
metadata: WireRevisionstoreMetadata,
}
impl ToWire for FileEntry {
type Wire = WireFileEntry;
fn to_wire(self) -> Self::Wire {
WireFileEntry {
key: self.key.to_wire(),
data: self.data,
parents: self.parents.to_wire(),
metadata: self.metadata.to_wire(),
}
}
}
impl ToApi for WireFileEntry {
type Api = FileEntry;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(FileEntry {
key: self.key.to_api()?,
data: self.data,
parents: self.parents.to_api()?,
metadata: self.metadata.to_api()?,
})
}
}
#[derive(Clone, Default, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct WireFileRequest {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub keys: Vec<WireKey>,
}
impl ToWire for FileRequest {
type Wire = WireFileRequest;
fn to_wire(self) -> Self::Wire {
WireFileRequest {
keys: self.keys.to_wire(),
}
}
}
impl ToApi for WireFileRequest {
type Api = FileRequest;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(FileRequest {
keys: self.keys.to_api()?,
})
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireFileEntry {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
let bytes: Vec<u8> = Arbitrary::arbitrary(g);
Self {
key: Arbitrary::arbitrary(g),
data: Bytes::from(bytes),
parents: Arbitrary::arbitrary(g),
metadata: Arbitrary::arbitrary(g),
}
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireFileRequest {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Self {
keys: Arbitrary::arbitrary(g),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::wire::tests::{check_serialize_roundtrip, check_wire_roundtrip};
use quickcheck::quickcheck;
quickcheck! {
fn test_request_roundtrip_serialize(v: WireFileRequest) -> bool {
check_serialize_roundtrip(v)
}
fn test_request_roundtrip_wire(v: FileRequest) -> bool {
check_wire_roundtrip(v)
}
fn test_entry_roundtrip_serialize(v: WireFileEntry) -> bool {
check_serialize_roundtrip(v)
}
fn test_entry_roundtrip_wire(v: FileEntry) -> bool {
check_wire_roundtrip(v)
}
}
}

View File

@ -0,0 +1,405 @@
/*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This software may be used and distributed according to the terms of the
* GNU General Public License version 2.
*/
//! This module contains wire representation structs for external types we'd
//! like to avoid explicitly depending on. Types will be added as they are
//! used.
//!
//! These types should all be `pub(crate)`. They're used extensively inside the
//! crate, but should never appear outside it. The methods on the request /
//! response objects should accept and return the public types from
//! `eden/scm/lib/types`.
//!
//! To maintain wire-protocol compatibility, we have some important conventions
//! and requirements for all types defined inside this module:
//!
//! 1. Every field should be renamed to a unique natural number using
//! `#[serde(rename = "0")]`. New fields should never re-use a field identifier
//! that has been used before. If a field changes semantically, it should be
//! considered a new field, and be given a new identifier.
//!
//! 2. Every enum should have an "Unknown" variant as the last variant in the
//! enum. This variant should be annotated with `#[serde(other, rename = "0")]`
//!
//! 3. When practical, fields should be annotated with
//! `#[serde(default, skip_serializing_if = "is_default")` to save space on the
//! wire. Do not use `#[serde(default)]` on the container.
//!
//! 4. All fields should be wrapped in `Option` or in a container that may be
//! empty, such as `Vec`. If an empty container has special semantics (other
//! than ignoring the field), please wrap that field in an `Option` as well to
//! distinguish between "empty" and "not present".
pub mod complete_tree;
pub mod file;
pub mod tree;
pub use crate::wire::{
complete_tree::WireCompleteTreeRequest,
file::{WireFileEntry, WireFileRequest},
tree::{WireTreeEntry, WireTreeRequest},
};
use std::convert::Infallible;
use std::fmt;
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
use serde_derive::{Deserialize, Serialize};
use thiserror::Error;
use revisionstore_types::Metadata as RevisionstoreMetadata;
use types::{path::ParseError as RepoPathParseError, HgId, Key, Parents, RepoPathBuf};
#[derive(Debug, Error)]
#[error("Failed to convert from wire to API representation")]
pub enum WireToApiConversionError {
UnrecognizedEnumVariant,
PathValidationError(RepoPathParseError),
}
impl From<Infallible> for WireToApiConversionError {
fn from(v: Infallible) -> Self {
match v {}
}
}
impl From<RepoPathParseError> for WireToApiConversionError {
fn from(v: RepoPathParseError) -> Self {
WireToApiConversionError::PathValidationError(v)
}
}
/// Convert from an EdenAPI API type to Wire type
pub trait ToWire: Sized {
type Wire: serde::Serialize + serde::de::DeserializeOwned + std::fmt::Debug + ToApi<Api = Self>;
fn to_wire(self) -> Self::Wire;
}
/// Covnert from an EdenAPI Wire type to API type
pub trait ToApi: Sized {
type Api;
type Error: Into<WireToApiConversionError> + fmt::Debug;
fn to_api(self) -> Result<Self::Api, Self::Error>;
}
impl<A: ToWire> ToWire for Vec<A> {
type Wire = Vec<<A as ToWire>::Wire>;
fn to_wire(self) -> Self::Wire {
let mut out = Vec::with_capacity(self.len());
for v in self.into_iter() {
out.push(v.to_wire())
}
out
}
}
impl<W: ToApi> ToApi for Vec<W> {
type Api = Vec<<W as ToApi>::Api>;
type Error = <W as ToApi>::Error;
fn to_api(self) -> Result<Self::Api, Self::Error> {
let mut out = Vec::with_capacity(self.len());
for v in self.into_iter() {
out.push(v.to_api()?)
}
Ok(out)
}
}
#[derive(Clone, Copy, Default, PartialEq, Eq, Serialize, Deserialize)]
pub struct WireHgId(
#[serde(rename = "0", default, skip_serializing_if = "is_default")] [u8; WireHgId::len()],
);
impl WireHgId {
const fn len() -> usize {
20
}
}
impl ToWire for HgId {
type Wire = WireHgId;
fn to_wire(self) -> Self::Wire {
WireHgId(self.into_byte_array())
}
}
impl ToApi for WireHgId {
type Api = HgId;
type Error = Infallible;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(HgId::from_byte_array(self.0))
}
}
impl fmt::Display for WireHgId {
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
match self.to_api() {
Ok(api) => fmt::Display::fmt(&api, fmt),
Err(_) => Err(fmt::Error),
}
}
}
impl fmt::Debug for WireHgId {
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
match self.to_api() {
Ok(api) => write!(fmt, "WireHgId({:?})", &api.to_hex()),
Err(_) => Err(fmt::Error),
}
}
}
#[derive(Clone, Default, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct WireRepoPathBuf(
#[serde(rename = "0", default, skip_serializing_if = "is_default")] String,
);
impl ToWire for RepoPathBuf {
type Wire = WireRepoPathBuf;
fn to_wire(self) -> Self::Wire {
WireRepoPathBuf(self.into_string())
}
}
impl ToApi for WireRepoPathBuf {
type Api = RepoPathBuf;
type Error = RepoPathParseError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(RepoPathBuf::from_string(self.0)?)
}
}
#[derive(Clone, Default, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct WireKey {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
path: WireRepoPathBuf,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
hgid: WireHgId,
}
impl ToWire for Key {
type Wire = WireKey;
fn to_wire(self) -> Self::Wire {
WireKey {
path: self.path.to_wire(),
hgid: self.hgid.to_wire(),
}
}
}
impl ToApi for WireKey {
type Api = Key;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(Key {
path: self.path.to_api()?,
hgid: self.hgid.to_api()?,
})
}
}
#[derive(Copy, Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub enum WireParents {
#[serde(rename = "1")]
None,
#[serde(rename = "2")]
One(WireHgId),
#[serde(rename = "3")]
Two(WireHgId, WireHgId),
#[serde(other, rename = "0")]
Unknown,
}
impl ToWire for Parents {
type Wire = WireParents;
fn to_wire(self) -> Self::Wire {
use Parents::*;
match self {
None => WireParents::None,
One(id) => WireParents::One(id.to_wire()),
Two(id1, id2) => WireParents::Two(id1.to_wire(), id2.to_wire()),
}
}
}
impl ToApi for WireParents {
type Api = Parents;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
use WireParents::*;
Ok(match self {
Unknown => {
return Err(WireToApiConversionError::UnrecognizedEnumVariant);
}
None => Parents::None,
One(id) => Parents::One(id.to_api()?),
Two(id1, id2) => Parents::Two(id1.to_api()?, id2.to_api()?),
})
}
}
impl Default for WireParents {
fn default() -> Self {
WireParents::None
}
}
#[derive(Clone, Copy, Default, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct WireRevisionstoreMetadata {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
size: Option<u64>,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
flags: Option<u64>,
}
impl ToWire for RevisionstoreMetadata {
type Wire = WireRevisionstoreMetadata;
fn to_wire(self) -> Self::Wire {
WireRevisionstoreMetadata {
size: self.size,
flags: self.flags,
}
}
}
impl ToApi for WireRevisionstoreMetadata {
type Api = RevisionstoreMetadata;
type Error = Infallible;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(RevisionstoreMetadata {
size: self.size,
flags: self.flags,
})
}
}
fn is_default<T: Default + PartialEq>(v: &T) -> bool {
v == &T::default()
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireHgId {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
HgId::arbitrary(g).to_wire()
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireRepoPathBuf {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
RepoPathBuf::arbitrary(g).to_wire()
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireKey {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Key::arbitrary(g).to_wire()
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireParents {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Parents::arbitrary(g).to_wire()
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireRevisionstoreMetadata {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Self {
size: Arbitrary::arbitrary(g),
flags: Arbitrary::arbitrary(g),
}
}
}
#[cfg(test)]
pub mod tests {
use super::*;
use quickcheck::quickcheck;
pub fn check_serialize_roundtrip<
T: serde::Serialize + serde::de::DeserializeOwned + Clone + PartialEq,
>(
original: T,
) -> bool {
let serial = serde_cbor::to_vec(&original).unwrap();
let roundtrip = serde_cbor::from_slice(&serial).unwrap();
original == roundtrip
}
pub fn check_wire_roundtrip<T: ToWire + Clone + PartialEq>(original: T) -> bool {
let wire = original.clone().to_wire();
let roundtrip = wire.to_api().unwrap();
original == roundtrip
}
quickcheck! {
fn test_hgid_roundtrip_serialize(v: WireHgId) -> bool {
check_serialize_roundtrip(v)
}
fn test_hgid_roundtrip_wire(v: HgId) -> bool {
check_wire_roundtrip(v)
}
fn test_key_roundtrip_serialize(v: WireKey) -> bool {
check_serialize_roundtrip(v)
}
fn test_key_roundtrip_wire(v: Key) -> bool {
check_wire_roundtrip(v)
}
fn test_path_roundtrip_serialize(v: WireRepoPathBuf) -> bool {
check_serialize_roundtrip(v)
}
fn test_path_roundtrip_wire(v: RepoPathBuf) -> bool {
check_wire_roundtrip(v)
}
fn test_parents_roundtrip_serialize(v: WireParents) -> bool {
check_serialize_roundtrip(v)
}
fn test_parents_roundtrip_wire(v: Parents) -> bool {
check_wire_roundtrip(v)
}
fn test_meta_roundtrip_serialize(v: WireRevisionstoreMetadata) -> bool {
check_serialize_roundtrip(v)
}
fn test_meta_roundtrip_wire(v: RevisionstoreMetadata) -> bool {
check_wire_roundtrip(v)
}
}
}

View File

@ -0,0 +1,137 @@
/*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This software may be used and distributed according to the terms of the
* GNU General Public License version 2.
*/
use bytes::Bytes;
#[cfg(any(test, feature = "for-tests"))]
use quickcheck::Arbitrary;
use serde_derive::{Deserialize, Serialize};
use crate::{
tree::{TreeEntry, TreeRequest},
wire::{
is_default, ToApi, ToWire, WireKey, WireParents, WireRevisionstoreMetadata,
WireToApiConversionError,
},
};
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
pub struct WireTreeEntry {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
key: WireKey,
#[serde(rename = "1", default, skip_serializing_if = "is_default")]
data: Bytes,
#[serde(rename = "2", default, skip_serializing_if = "is_default")]
parents: WireParents,
#[serde(rename = "3", default, skip_serializing_if = "is_default")]
metadata: WireRevisionstoreMetadata,
}
impl ToWire for TreeEntry {
type Wire = WireTreeEntry;
fn to_wire(self) -> Self::Wire {
WireTreeEntry {
key: self.key.to_wire(),
data: self.data,
parents: self.parents.to_wire(),
metadata: self.metadata.to_wire(),
}
}
}
impl ToApi for WireTreeEntry {
type Api = TreeEntry;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(TreeEntry {
key: self.key.to_api()?,
data: self.data,
parents: self.parents.to_api()?,
metadata: self.metadata.to_api()?,
})
}
}
#[derive(Clone, Debug, Default, Serialize, Deserialize, Eq, PartialEq)]
pub struct WireTreeRequest {
#[serde(rename = "0", default, skip_serializing_if = "is_default")]
pub keys: Vec<WireKey>,
}
impl ToWire for TreeRequest {
type Wire = WireTreeRequest;
fn to_wire(self) -> Self::Wire {
WireTreeRequest {
keys: self.keys.to_wire(),
}
}
}
impl ToApi for WireTreeRequest {
type Api = TreeRequest;
type Error = WireToApiConversionError;
fn to_api(self) -> Result<Self::Api, Self::Error> {
Ok(TreeRequest {
keys: self.keys.to_api()?,
})
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireTreeEntry {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
let bytes: Vec<u8> = Arbitrary::arbitrary(g);
Self {
key: Arbitrary::arbitrary(g),
data: Bytes::from(bytes),
parents: Arbitrary::arbitrary(g),
metadata: Arbitrary::arbitrary(g),
}
}
}
#[cfg(any(test, feature = "for-tests"))]
impl Arbitrary for WireTreeRequest {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Self {
keys: Arbitrary::arbitrary(g),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::wire::tests::{check_serialize_roundtrip, check_wire_roundtrip};
use quickcheck::quickcheck;
quickcheck! {
fn test_request_roundtrip_serialize(v: WireTreeRequest) -> bool {
check_serialize_roundtrip(v)
}
fn test_request_roundtrip_wire(v: TreeRequest) -> bool {
check_wire_roundtrip(v)
}
fn test_entry_roundtrip_serialize(v: WireTreeEntry) -> bool {
check_serialize_roundtrip(v)
}
fn test_entry_roundtrip_wire(v: TreeEntry) -> bool {
check_wire_roundtrip(v)
}
}
}

View File

@ -9,11 +9,13 @@ include = ["**/*.rs"]
[lib]
path = "lib.rs"
[features]
default = ["for-tests"]
for-tests = []
[dependencies]
anyhow = "1.0"
byteorder = "1.3"
quickcheck = "0.9"
serde = { version = "1.0", features = ["derive", "rc"] }
serde_derive = "1.0"
[dev-dependencies]
quickcheck = "0.9"

View File

@ -17,6 +17,16 @@ pub struct Metadata {
pub flags: Option<u64>,
}
#[cfg(any(test, feature = "for-tests"))]
impl quickcheck::Arbitrary for Metadata {
fn arbitrary<G: quickcheck::Gen>(g: &mut G) -> Self {
Self {
size: quickcheck::Arbitrary::arbitrary(g),
flags: quickcheck::Arbitrary::arbitrary(g),
}
}
}
impl Metadata {
pub const LFS_FLAG: u64 = 0x2000;

View File

@ -95,6 +95,10 @@ impl HgId {
HgId(bytes)
}
pub fn into_byte_array(self) -> [u8; HgId::len()] {
self.0
}
pub fn to_hex(&self) -> String {
to_hex(self.0.as_ref())
}