2019-10-12 11:53:11 +00:00
|
|
|
use std::collections::HashSet;
|
2020-06-25 10:23:30 +00:00
|
|
|
use std::os::unix::fs::OpenOptionsExt;
|
2019-10-12 11:53:11 +00:00
|
|
|
use std::sync::atomic::{AtomicUsize, Ordering};
|
|
|
|
use std::sync::{Arc, Mutex};
|
|
|
|
|
2020-07-07 13:20:20 +00:00
|
|
|
use anyhow::{bail, format_err, Error};
|
2019-12-17 09:52:07 +00:00
|
|
|
use futures::future::AbortHandle;
|
2021-03-24 16:12:05 +00:00
|
|
|
use futures::stream::Stream;
|
|
|
|
use futures::*;
|
2019-10-12 11:53:11 +00:00
|
|
|
use serde_json::{json, Value};
|
|
|
|
use tokio::io::AsyncReadExt;
|
|
|
|
use tokio::sync::{mpsc, oneshot};
|
2021-01-11 08:50:04 +00:00
|
|
|
use tokio_stream::wrappers::ReceiverStream;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
use proxmox::tools::digest_to_hex;
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
use super::merge_known_chunks::{MergeKnownChunks, MergedChunkInfo};
|
2019-10-12 11:53:11 +00:00
|
|
|
use crate::backup::*;
|
2020-07-23 07:45:48 +00:00
|
|
|
use crate::tools::format::HumanByte;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
use super::{H2Client, HttpClient};
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
pub struct BackupWriter {
|
|
|
|
h2: H2Client,
|
2019-12-17 09:52:07 +00:00
|
|
|
abort: AbortHandle,
|
2020-02-24 11:48:40 +00:00
|
|
|
verbose: bool,
|
2020-06-25 10:23:30 +00:00
|
|
|
crypt_config: Option<Arc<CryptConfig>>,
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
impl Drop for BackupWriter {
|
|
|
|
fn drop(&mut self) {
|
2019-12-17 09:52:07 +00:00
|
|
|
self.abort.abort();
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
pub struct BackupStats {
|
|
|
|
pub size: u64,
|
|
|
|
pub csum: [u8; 32],
|
|
|
|
}
|
|
|
|
|
2021-01-25 13:42:52 +00:00
|
|
|
/// Options for uploading blobs/streams to the server
|
|
|
|
#[derive(Default, Clone)]
|
|
|
|
pub struct UploadOptions {
|
|
|
|
pub previous_manifest: Option<Arc<BackupManifest>>,
|
|
|
|
pub compress: bool,
|
|
|
|
pub encrypt: bool,
|
|
|
|
pub fixed_size: Option<u64>,
|
|
|
|
}
|
|
|
|
|
2021-03-24 16:17:18 +00:00
|
|
|
struct UploadStats {
|
|
|
|
chunk_count: usize,
|
|
|
|
chunk_reused: usize,
|
|
|
|
size: usize,
|
|
|
|
size_reused: usize,
|
|
|
|
duration: std::time::Duration,
|
|
|
|
csum: [u8; 32],
|
|
|
|
}
|
|
|
|
|
2020-10-14 09:33:45 +00:00
|
|
|
type UploadQueueSender = mpsc::Sender<(MergedChunkInfo, Option<h2::client::ResponseFuture>)>;
|
|
|
|
type UploadResultReceiver = oneshot::Receiver<Result<(), Error>>;
|
|
|
|
|
2019-10-12 11:53:11 +00:00
|
|
|
impl BackupWriter {
|
2021-03-24 16:12:05 +00:00
|
|
|
fn new(
|
|
|
|
h2: H2Client,
|
|
|
|
abort: AbortHandle,
|
|
|
|
crypt_config: Option<Arc<CryptConfig>>,
|
|
|
|
verbose: bool,
|
|
|
|
) -> Arc<Self> {
|
|
|
|
Arc::new(Self {
|
|
|
|
h2,
|
|
|
|
abort,
|
|
|
|
crypt_config,
|
|
|
|
verbose,
|
|
|
|
})
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
2021-01-25 13:42:59 +00:00
|
|
|
// FIXME: extract into (flattened) parameter struct?
|
|
|
|
#[allow(clippy::too_many_arguments)]
|
2019-10-12 11:53:11 +00:00
|
|
|
pub async fn start(
|
|
|
|
client: HttpClient,
|
2020-06-25 10:23:30 +00:00
|
|
|
crypt_config: Option<Arc<CryptConfig>>,
|
2019-10-12 11:53:11 +00:00
|
|
|
datastore: &str,
|
|
|
|
backup_type: &str,
|
|
|
|
backup_id: &str,
|
2020-09-12 13:10:47 +00:00
|
|
|
backup_time: i64,
|
2019-10-12 11:53:11 +00:00
|
|
|
debug: bool,
|
2021-03-24 16:12:05 +00:00
|
|
|
benchmark: bool,
|
2019-10-12 11:53:11 +00:00
|
|
|
) -> Result<Arc<BackupWriter>, Error> {
|
|
|
|
let param = json!({
|
|
|
|
"backup-type": backup_type,
|
|
|
|
"backup-id": backup_id,
|
2020-09-12 13:10:47 +00:00
|
|
|
"backup-time": backup_time,
|
2019-10-12 11:53:11 +00:00
|
|
|
"store": datastore,
|
2020-09-02 09:41:22 +00:00
|
|
|
"debug": debug,
|
|
|
|
"benchmark": benchmark
|
2019-10-12 11:53:11 +00:00
|
|
|
});
|
|
|
|
|
|
|
|
let req = HttpClient::request_builder(
|
2021-03-24 16:12:05 +00:00
|
|
|
client.server(),
|
|
|
|
client.port(),
|
|
|
|
"GET",
|
|
|
|
"/api2/json/backup",
|
|
|
|
Some(param),
|
|
|
|
)
|
|
|
|
.unwrap();
|
|
|
|
|
|
|
|
let (h2, abort) = client
|
|
|
|
.start_h2_connection(req, String::from(PROXMOX_BACKUP_PROTOCOL_ID_V1!()))
|
|
|
|
.await?;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
Ok(BackupWriter::new(h2, abort, crypt_config, debug))
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
pub async fn get(&self, path: &str, param: Option<Value>) -> Result<Value, Error> {
|
2019-10-12 11:53:11 +00:00
|
|
|
self.h2.get(path, param).await
|
|
|
|
}
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
pub async fn put(&self, path: &str, param: Option<Value>) -> Result<Value, Error> {
|
2019-10-12 11:53:11 +00:00
|
|
|
self.h2.put(path, param).await
|
|
|
|
}
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
pub async fn post(&self, path: &str, param: Option<Value>) -> Result<Value, Error> {
|
2019-10-12 11:53:11 +00:00
|
|
|
self.h2.post(path, param).await
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn upload_post(
|
|
|
|
&self,
|
|
|
|
path: &str,
|
|
|
|
param: Option<Value>,
|
|
|
|
content_type: &str,
|
|
|
|
data: Vec<u8>,
|
|
|
|
) -> Result<Value, Error> {
|
2021-03-24 16:12:05 +00:00
|
|
|
self.h2
|
|
|
|
.upload("POST", path, param, content_type, data)
|
|
|
|
.await
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn send_upload_request(
|
|
|
|
&self,
|
|
|
|
method: &str,
|
|
|
|
path: &str,
|
|
|
|
param: Option<Value>,
|
|
|
|
content_type: &str,
|
|
|
|
data: Vec<u8>,
|
|
|
|
) -> Result<h2::client::ResponseFuture, Error> {
|
2021-03-24 16:12:05 +00:00
|
|
|
let request =
|
|
|
|
H2Client::request_builder("localhost", method, path, param, Some(content_type))
|
|
|
|
.unwrap();
|
|
|
|
let response_future = self
|
|
|
|
.h2
|
|
|
|
.send_request(request, Some(bytes::Bytes::from(data.clone())))
|
|
|
|
.await?;
|
2019-10-12 11:53:11 +00:00
|
|
|
Ok(response_future)
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn upload_put(
|
|
|
|
&self,
|
|
|
|
path: &str,
|
|
|
|
param: Option<Value>,
|
|
|
|
content_type: &str,
|
|
|
|
data: Vec<u8>,
|
|
|
|
) -> Result<Value, Error> {
|
|
|
|
self.h2.upload("PUT", path, param, content_type, data).await
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn finish(self: Arc<Self>) -> Result<(), Error> {
|
|
|
|
let h2 = self.h2.clone();
|
|
|
|
|
|
|
|
h2.post("finish", None)
|
|
|
|
.map_ok(move |_| {
|
2019-12-17 09:52:07 +00:00
|
|
|
self.abort.abort();
|
2019-10-12 11:53:11 +00:00
|
|
|
})
|
|
|
|
.await
|
|
|
|
}
|
|
|
|
|
2019-11-08 09:34:16 +00:00
|
|
|
pub fn cancel(&self) {
|
2019-12-17 09:52:07 +00:00
|
|
|
self.abort.abort();
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn upload_blob<R: std::io::Read>(
|
|
|
|
&self,
|
|
|
|
mut reader: R,
|
|
|
|
file_name: &str,
|
2021-03-24 16:12:05 +00:00
|
|
|
) -> Result<BackupStats, Error> {
|
2019-10-12 11:53:11 +00:00
|
|
|
let mut raw_data = Vec::new();
|
|
|
|
// fixme: avoid loading into memory
|
|
|
|
reader.read_to_end(&mut raw_data)?;
|
|
|
|
|
|
|
|
let csum = openssl::sha::sha256(&raw_data);
|
|
|
|
let param = json!({"encoded-size": raw_data.len(), "file-name": file_name });
|
|
|
|
let size = raw_data.len() as u64;
|
2021-03-24 16:12:05 +00:00
|
|
|
let _value = self
|
|
|
|
.h2
|
|
|
|
.upload(
|
|
|
|
"POST",
|
|
|
|
"blob",
|
|
|
|
Some(param),
|
|
|
|
"application/octet-stream",
|
|
|
|
raw_data,
|
|
|
|
)
|
|
|
|
.await?;
|
2019-10-12 11:53:11 +00:00
|
|
|
Ok(BackupStats { size, csum })
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn upload_blob_from_data(
|
|
|
|
&self,
|
|
|
|
data: Vec<u8>,
|
|
|
|
file_name: &str,
|
2021-01-25 13:42:52 +00:00
|
|
|
options: UploadOptions,
|
2020-07-07 13:20:20 +00:00
|
|
|
) -> Result<BackupStats, Error> {
|
2021-01-25 13:42:52 +00:00
|
|
|
let blob = match (options.encrypt, &self.crypt_config) {
|
2021-03-24 16:12:05 +00:00
|
|
|
(false, _) => DataBlob::encode(&data, None, options.compress)?,
|
|
|
|
(true, None) => bail!("requested encryption without a crypt config"),
|
|
|
|
(true, Some(crypt_config)) => {
|
|
|
|
DataBlob::encode(&data, Some(crypt_config), options.compress)?
|
|
|
|
}
|
2019-10-12 11:53:11 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
let raw_data = blob.into_inner();
|
|
|
|
let size = raw_data.len() as u64;
|
|
|
|
|
|
|
|
let csum = openssl::sha::sha256(&raw_data);
|
|
|
|
let param = json!({"encoded-size": size, "file-name": file_name });
|
2021-03-24 16:12:05 +00:00
|
|
|
let _value = self
|
|
|
|
.h2
|
|
|
|
.upload(
|
|
|
|
"POST",
|
|
|
|
"blob",
|
|
|
|
Some(param),
|
|
|
|
"application/octet-stream",
|
|
|
|
raw_data,
|
|
|
|
)
|
|
|
|
.await?;
|
2019-10-12 11:53:11 +00:00
|
|
|
Ok(BackupStats { size, csum })
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn upload_blob_from_file<P: AsRef<std::path::Path>>(
|
|
|
|
&self,
|
|
|
|
src_path: P,
|
|
|
|
file_name: &str,
|
2021-01-25 13:42:52 +00:00
|
|
|
options: UploadOptions,
|
2020-07-08 12:06:50 +00:00
|
|
|
) -> Result<BackupStats, Error> {
|
2019-10-12 11:53:11 +00:00
|
|
|
let src_path = src_path.as_ref();
|
|
|
|
|
|
|
|
let mut file = tokio::fs::File::open(src_path)
|
|
|
|
.await
|
|
|
|
.map_err(|err| format_err!("unable to open file {:?} - {}", src_path, err))?;
|
|
|
|
|
|
|
|
let mut contents = Vec::new();
|
|
|
|
|
|
|
|
file.read_to_end(&mut contents)
|
|
|
|
.await
|
|
|
|
.map_err(|err| format_err!("unable to read file {:?} - {}", src_path, err))?;
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
self.upload_blob_from_data(contents, file_name, options)
|
|
|
|
.await
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn upload_stream(
|
|
|
|
&self,
|
|
|
|
archive_name: &str,
|
|
|
|
stream: impl Stream<Item = Result<bytes::BytesMut, Error>>,
|
2021-01-25 13:42:52 +00:00
|
|
|
options: UploadOptions,
|
2019-10-12 11:53:11 +00:00
|
|
|
) -> Result<BackupStats, Error> {
|
|
|
|
let known_chunks = Arc::new(Mutex::new(HashSet::new()));
|
|
|
|
|
|
|
|
let mut param = json!({ "archive-name": archive_name });
|
2021-01-25 13:42:52 +00:00
|
|
|
let prefix = if let Some(size) = options.fixed_size {
|
2019-10-12 11:53:11 +00:00
|
|
|
param["size"] = size.into();
|
2021-01-25 13:42:52 +00:00
|
|
|
"fixed"
|
|
|
|
} else {
|
|
|
|
"dynamic"
|
|
|
|
};
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2021-01-25 13:42:52 +00:00
|
|
|
if options.encrypt && self.crypt_config.is_none() {
|
2020-07-08 12:06:50 +00:00
|
|
|
bail!("requested encryption without a crypt config");
|
|
|
|
}
|
|
|
|
|
2019-10-12 11:53:11 +00:00
|
|
|
let index_path = format!("{}_index", prefix);
|
|
|
|
let close_path = format!("{}_close", prefix);
|
|
|
|
|
2021-01-25 13:42:52 +00:00
|
|
|
if let Some(manifest) = options.previous_manifest {
|
2020-06-25 10:23:30 +00:00
|
|
|
// try, but ignore errors
|
|
|
|
match archive_type(archive_name) {
|
|
|
|
Ok(ArchiveType::FixedIndex) => {
|
2021-03-24 16:12:05 +00:00
|
|
|
let _ = self
|
|
|
|
.download_previous_fixed_index(
|
|
|
|
archive_name,
|
|
|
|
&manifest,
|
|
|
|
known_chunks.clone(),
|
|
|
|
)
|
|
|
|
.await;
|
2020-06-25 10:23:30 +00:00
|
|
|
}
|
|
|
|
Ok(ArchiveType::DynamicIndex) => {
|
2021-03-24 16:12:05 +00:00
|
|
|
let _ = self
|
|
|
|
.download_previous_dynamic_index(
|
|
|
|
archive_name,
|
|
|
|
&manifest,
|
|
|
|
known_chunks.clone(),
|
|
|
|
)
|
|
|
|
.await;
|
2020-06-25 10:23:30 +00:00
|
|
|
}
|
|
|
|
_ => { /* do nothing */ }
|
|
|
|
}
|
|
|
|
}
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
let wid = self
|
|
|
|
.h2
|
|
|
|
.post(&index_path, Some(param))
|
|
|
|
.await?
|
|
|
|
.as_u64()
|
|
|
|
.unwrap();
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2021-03-24 16:17:18 +00:00
|
|
|
let upload_stats = Self::upload_chunk_info_stream(
|
|
|
|
self.h2.clone(),
|
|
|
|
wid,
|
|
|
|
stream,
|
|
|
|
&prefix,
|
|
|
|
known_chunks.clone(),
|
|
|
|
if options.encrypt {
|
|
|
|
self.crypt_config.clone()
|
|
|
|
} else {
|
|
|
|
None
|
|
|
|
},
|
|
|
|
options.compress,
|
|
|
|
self.verbose,
|
|
|
|
)
|
|
|
|
.await?;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2021-03-24 16:17:18 +00:00
|
|
|
let uploaded = upload_stats.size - upload_stats.size_reused;
|
|
|
|
let vsize_h: HumanByte = upload_stats.size.into();
|
2020-07-23 07:45:48 +00:00
|
|
|
let archive = if self.verbose {
|
|
|
|
archive_name.to_string()
|
|
|
|
} else {
|
2020-10-14 09:33:45 +00:00
|
|
|
crate::tools::format::strip_server_file_extension(archive_name)
|
2020-07-23 07:45:48 +00:00
|
|
|
};
|
|
|
|
if archive_name != CATALOG_NAME {
|
2021-03-24 16:12:05 +00:00
|
|
|
let speed: HumanByte =
|
2021-03-24 16:17:18 +00:00
|
|
|
((uploaded * 1_000_000) / (upload_stats.duration.as_micros() as usize)).into();
|
2020-07-23 07:45:48 +00:00
|
|
|
let uploaded: HumanByte = uploaded.into();
|
2021-03-24 16:12:05 +00:00
|
|
|
println!(
|
|
|
|
"{}: had to upload {} of {} in {:.2}s, average speed {}/s).",
|
|
|
|
archive,
|
|
|
|
uploaded,
|
|
|
|
vsize_h,
|
2021-03-24 16:17:18 +00:00
|
|
|
upload_stats.duration.as_secs_f64(),
|
2021-03-24 16:12:05 +00:00
|
|
|
speed
|
|
|
|
);
|
2020-07-23 07:45:48 +00:00
|
|
|
} else {
|
|
|
|
println!("Uploaded backup catalog ({})", vsize_h);
|
|
|
|
}
|
|
|
|
|
2021-03-24 16:17:18 +00:00
|
|
|
if upload_stats.size_reused > 0 && upload_stats.size > 1024 * 1024 {
|
|
|
|
let reused_percent = upload_stats.size_reused as f64 * 100. / upload_stats.size as f64;
|
|
|
|
let reused: HumanByte = upload_stats.size_reused.into();
|
2021-03-24 16:12:05 +00:00
|
|
|
println!(
|
|
|
|
"{}: backup was done incrementally, reused {} ({:.1}%)",
|
|
|
|
archive, reused, reused_percent
|
|
|
|
);
|
2020-07-23 07:45:48 +00:00
|
|
|
}
|
2021-03-24 16:17:18 +00:00
|
|
|
if self.verbose && upload_stats.chunk_count > 0 {
|
2021-03-24 16:12:05 +00:00
|
|
|
println!(
|
|
|
|
"{}: Reused {} from {} chunks.",
|
2021-03-24 16:17:18 +00:00
|
|
|
archive, upload_stats.chunk_reused, upload_stats.chunk_count
|
2021-03-24 16:12:05 +00:00
|
|
|
);
|
|
|
|
println!(
|
|
|
|
"{}: Average chunk size was {}.",
|
|
|
|
archive,
|
2021-03-24 16:17:18 +00:00
|
|
|
HumanByte::from(upload_stats.size / upload_stats.chunk_count)
|
2021-03-24 16:12:05 +00:00
|
|
|
);
|
|
|
|
println!(
|
|
|
|
"{}: Average time per request: {} microseconds.",
|
|
|
|
archive,
|
2021-03-24 16:17:18 +00:00
|
|
|
(upload_stats.duration.as_micros()) / (upload_stats.chunk_count as u128)
|
2021-03-24 16:12:05 +00:00
|
|
|
);
|
2020-02-24 12:24:46 +00:00
|
|
|
}
|
|
|
|
|
2019-10-12 11:53:11 +00:00
|
|
|
let param = json!({
|
|
|
|
"wid": wid ,
|
2021-03-24 16:17:18 +00:00
|
|
|
"chunk-count": upload_stats.chunk_count,
|
|
|
|
"size": upload_stats.size,
|
|
|
|
"csum": proxmox::tools::digest_to_hex(&upload_stats.csum),
|
2019-10-12 11:53:11 +00:00
|
|
|
});
|
|
|
|
let _value = self.h2.post(&close_path, Some(param)).await?;
|
|
|
|
Ok(BackupStats {
|
2021-03-24 16:17:18 +00:00
|
|
|
size: upload_stats.size as u64,
|
|
|
|
csum: upload_stats.csum,
|
2019-10-12 11:53:11 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
fn response_queue(
|
|
|
|
verbose: bool,
|
|
|
|
) -> (
|
2019-10-12 11:53:11 +00:00
|
|
|
mpsc::Sender<h2::client::ResponseFuture>,
|
2021-03-24 16:12:05 +00:00
|
|
|
oneshot::Receiver<Result<(), Error>>,
|
2019-10-12 11:53:11 +00:00
|
|
|
) {
|
|
|
|
let (verify_queue_tx, verify_queue_rx) = mpsc::channel(100);
|
|
|
|
let (verify_result_tx, verify_result_rx) = oneshot::channel();
|
|
|
|
|
2019-12-12 14:27:07 +00:00
|
|
|
// FIXME: check if this works as expected as replacement for the combinator below?
|
|
|
|
// tokio::spawn(async move {
|
|
|
|
// let result: Result<(), Error> = (async move {
|
|
|
|
// while let Some(response) = verify_queue_rx.recv().await {
|
|
|
|
// match H2Client::h2api_response(response.await?).await {
|
|
|
|
// Ok(result) => println!("RESPONSE: {:?}", result),
|
|
|
|
// Err(err) => bail!("pipelined request failed: {}", err),
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
// Ok(())
|
|
|
|
// }).await;
|
|
|
|
// let _ignore_closed_channel = verify_result_tx.send(result);
|
|
|
|
// });
|
|
|
|
// old code for reference?
|
|
|
|
tokio::spawn(
|
2021-01-11 08:50:04 +00:00
|
|
|
ReceiverStream::new(verify_queue_rx)
|
2019-10-12 11:53:11 +00:00
|
|
|
.map(Ok::<_, Error>)
|
2020-07-09 14:16:39 +00:00
|
|
|
.try_for_each(move |response: h2::client::ResponseFuture| {
|
2019-10-12 11:53:11 +00:00
|
|
|
response
|
|
|
|
.map_err(Error::from)
|
|
|
|
.and_then(H2Client::h2api_response)
|
2021-03-24 16:12:05 +00:00
|
|
|
.map_ok(move |result| {
|
|
|
|
if verbose {
|
|
|
|
println!("RESPONSE: {:?}", result)
|
|
|
|
}
|
|
|
|
})
|
2019-10-12 11:53:11 +00:00
|
|
|
.map_err(|err| format_err!("pipelined request failed: {}", err))
|
|
|
|
})
|
|
|
|
.map(|result| {
|
2021-03-24 16:12:05 +00:00
|
|
|
let _ignore_closed_channel = verify_result_tx.send(result);
|
|
|
|
}),
|
2019-10-12 11:53:11 +00:00
|
|
|
);
|
|
|
|
|
|
|
|
(verify_queue_tx, verify_result_rx)
|
|
|
|
}
|
|
|
|
|
2020-10-14 09:33:45 +00:00
|
|
|
fn append_chunk_queue(
|
|
|
|
h2: H2Client,
|
|
|
|
wid: u64,
|
|
|
|
path: String,
|
|
|
|
verbose: bool,
|
|
|
|
) -> (UploadQueueSender, UploadResultReceiver) {
|
2019-10-12 11:53:11 +00:00
|
|
|
let (verify_queue_tx, verify_queue_rx) = mpsc::channel(64);
|
|
|
|
let (verify_result_tx, verify_result_rx) = oneshot::channel();
|
|
|
|
|
2019-12-12 14:27:07 +00:00
|
|
|
// FIXME: async-block-ify this code!
|
|
|
|
tokio::spawn(
|
2021-01-11 08:50:04 +00:00
|
|
|
ReceiverStream::new(verify_queue_rx)
|
2019-10-12 11:53:11 +00:00
|
|
|
.map(Ok::<_, Error>)
|
|
|
|
.and_then(move |(merged_chunk_info, response): (MergedChunkInfo, Option<h2::client::ResponseFuture>)| {
|
|
|
|
match (response, merged_chunk_info) {
|
|
|
|
(Some(response), MergedChunkInfo::Known(list)) => {
|
|
|
|
future::Either::Left(
|
|
|
|
response
|
|
|
|
.map_err(Error::from)
|
|
|
|
.and_then(H2Client::h2api_response)
|
|
|
|
.and_then(move |_result| {
|
|
|
|
future::ok(MergedChunkInfo::Known(list))
|
|
|
|
})
|
|
|
|
)
|
|
|
|
}
|
|
|
|
(None, MergedChunkInfo::Known(list)) => {
|
|
|
|
future::Either::Right(future::ok(MergedChunkInfo::Known(list)))
|
|
|
|
}
|
|
|
|
_ => unreachable!(),
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.merge_known_chunks()
|
|
|
|
.and_then(move |merged_chunk_info| {
|
|
|
|
match merged_chunk_info {
|
|
|
|
MergedChunkInfo::Known(chunk_list) => {
|
|
|
|
let mut digest_list = vec![];
|
|
|
|
let mut offset_list = vec![];
|
|
|
|
for (offset, digest) in chunk_list {
|
|
|
|
digest_list.push(digest_to_hex(&digest));
|
|
|
|
offset_list.push(offset);
|
|
|
|
}
|
2020-02-24 11:48:40 +00:00
|
|
|
if verbose { println!("append chunks list len ({})", digest_list.len()); }
|
2019-10-12 11:53:11 +00:00
|
|
|
let param = json!({ "wid": wid, "digest-list": digest_list, "offset-list": offset_list });
|
|
|
|
let request = H2Client::request_builder("localhost", "PUT", &path, None, Some("application/json")).unwrap();
|
2019-12-12 14:27:07 +00:00
|
|
|
let param_data = bytes::Bytes::from(param.to_string().into_bytes());
|
2019-10-12 11:53:11 +00:00
|
|
|
let upload_data = Some(param_data);
|
2020-10-14 09:33:45 +00:00
|
|
|
h2.send_request(request, upload_data)
|
2019-10-12 11:53:11 +00:00
|
|
|
.and_then(move |response| {
|
|
|
|
response
|
|
|
|
.map_err(Error::from)
|
|
|
|
.and_then(H2Client::h2api_response)
|
|
|
|
.map_ok(|_| ())
|
|
|
|
})
|
|
|
|
.map_err(|err| format_err!("pipelined request failed: {}", err))
|
|
|
|
}
|
|
|
|
_ => unreachable!(),
|
|
|
|
}
|
|
|
|
})
|
|
|
|
.try_for_each(|_| future::ok(()))
|
|
|
|
.map(|result| {
|
|
|
|
let _ignore_closed_channel = verify_result_tx.send(result);
|
|
|
|
})
|
|
|
|
);
|
|
|
|
|
|
|
|
(verify_queue_tx, verify_result_rx)
|
|
|
|
}
|
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
pub async fn download_previous_fixed_index(
|
2019-10-12 11:53:11 +00:00
|
|
|
&self,
|
|
|
|
archive_name: &str,
|
2020-06-25 10:23:30 +00:00
|
|
|
manifest: &BackupManifest,
|
2021-03-24 16:12:05 +00:00
|
|
|
known_chunks: Arc<Mutex<HashSet<[u8; 32]>>>,
|
2020-06-25 10:23:30 +00:00
|
|
|
) -> Result<FixedIndexReader, Error> {
|
|
|
|
let mut tmpfile = std::fs::OpenOptions::new()
|
|
|
|
.write(true)
|
|
|
|
.read(true)
|
|
|
|
.custom_flags(libc::O_TMPFILE)
|
|
|
|
.open("/tmp")?;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
let param = json!({ "archive-name": archive_name });
|
2021-03-24 16:12:05 +00:00
|
|
|
self.h2
|
|
|
|
.download("previous", Some(param), &mut tmpfile)
|
|
|
|
.await?;
|
2020-06-25 10:23:30 +00:00
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
let index = FixedIndexReader::new(tmpfile).map_err(|err| {
|
|
|
|
format_err!("unable to read fixed index '{}' - {}", archive_name, err)
|
|
|
|
})?;
|
2020-06-25 10:23:30 +00:00
|
|
|
// Note: do not use values stored in index (not trusted) - instead, computed them again
|
|
|
|
let (csum, size) = index.compute_csum();
|
|
|
|
manifest.verify_file(archive_name, &csum, size)?;
|
|
|
|
|
|
|
|
// add index chunks to known chunks
|
|
|
|
let mut known_chunks = known_chunks.lock().unwrap();
|
|
|
|
for i in 0..index.index_count() {
|
|
|
|
known_chunks.insert(*index.index_digest(i).unwrap());
|
|
|
|
}
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
if self.verbose {
|
2021-03-24 16:12:05 +00:00
|
|
|
println!(
|
|
|
|
"{}: known chunks list length is {}",
|
|
|
|
archive_name,
|
|
|
|
index.index_count()
|
|
|
|
);
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
Ok(index)
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn download_previous_dynamic_index(
|
|
|
|
&self,
|
|
|
|
archive_name: &str,
|
|
|
|
manifest: &BackupManifest,
|
2021-03-24 16:12:05 +00:00
|
|
|
known_chunks: Arc<Mutex<HashSet<[u8; 32]>>>,
|
2020-06-25 10:23:30 +00:00
|
|
|
) -> Result<DynamicIndexReader, Error> {
|
|
|
|
let mut tmpfile = std::fs::OpenOptions::new()
|
|
|
|
.write(true)
|
|
|
|
.read(true)
|
|
|
|
.custom_flags(libc::O_TMPFILE)
|
|
|
|
.open("/tmp")?;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
let param = json!({ "archive-name": archive_name });
|
2021-03-24 16:12:05 +00:00
|
|
|
self.h2
|
|
|
|
.download("previous", Some(param), &mut tmpfile)
|
|
|
|
.await?;
|
2020-06-25 10:23:30 +00:00
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
let index = DynamicIndexReader::new(tmpfile).map_err(|err| {
|
|
|
|
format_err!("unable to read dynmamic index '{}' - {}", archive_name, err)
|
|
|
|
})?;
|
2020-06-25 10:23:30 +00:00
|
|
|
// Note: do not use values stored in index (not trusted) - instead, computed them again
|
|
|
|
let (csum, size) = index.compute_csum();
|
|
|
|
manifest.verify_file(archive_name, &csum, size)?;
|
|
|
|
|
|
|
|
// add index chunks to known chunks
|
|
|
|
let mut known_chunks = known_chunks.lock().unwrap();
|
|
|
|
for i in 0..index.index_count() {
|
|
|
|
known_chunks.insert(*index.index_digest(i).unwrap());
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
2020-02-24 11:48:40 +00:00
|
|
|
if self.verbose {
|
2021-03-24 16:12:05 +00:00
|
|
|
println!(
|
|
|
|
"{}: known chunks list length is {}",
|
|
|
|
archive_name,
|
|
|
|
index.index_count()
|
|
|
|
);
|
2020-02-24 11:48:40 +00:00
|
|
|
}
|
2020-02-24 10:33:00 +00:00
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
Ok(index)
|
|
|
|
}
|
|
|
|
|
2020-11-20 16:38:40 +00:00
|
|
|
/// Retrieve backup time of last backup
|
|
|
|
pub async fn previous_backup_time(&self) -> Result<Option<i64>, Error> {
|
|
|
|
let data = self.h2.get("previous_backup_time", None).await?;
|
2021-03-24 16:12:05 +00:00
|
|
|
serde_json::from_value(data).map_err(|err| {
|
|
|
|
format_err!(
|
|
|
|
"Failed to parse backup time value returned by server - {}",
|
|
|
|
err
|
|
|
|
)
|
|
|
|
})
|
2020-11-20 16:38:40 +00:00
|
|
|
}
|
|
|
|
|
2020-06-25 10:23:30 +00:00
|
|
|
/// Download backup manifest (index.json) of last backup
|
|
|
|
pub async fn download_previous_manifest(&self) -> Result<BackupManifest, Error> {
|
|
|
|
let mut raw_data = Vec::with_capacity(64 * 1024);
|
|
|
|
|
|
|
|
let param = json!({ "archive-name": MANIFEST_BLOB_NAME });
|
2021-03-24 16:12:05 +00:00
|
|
|
self.h2
|
|
|
|
.download("previous", Some(param), &mut raw_data)
|
|
|
|
.await?;
|
2020-06-25 10:23:30 +00:00
|
|
|
|
2020-07-28 08:23:16 +00:00
|
|
|
let blob = DataBlob::load_from_reader(&mut &raw_data[..])?;
|
2020-08-03 12:10:43 +00:00
|
|
|
// no expected digest available
|
|
|
|
let data = blob.decode(self.crypt_config.as_ref().map(Arc::as_ref), None)?;
|
2020-07-09 07:48:30 +00:00
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
let manifest =
|
|
|
|
BackupManifest::from_data(&data[..], self.crypt_config.as_ref().map(Arc::as_ref))?;
|
2020-06-25 10:23:30 +00:00
|
|
|
|
|
|
|
Ok(manifest)
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
2020-10-14 09:33:45 +00:00
|
|
|
// We have no `self` here for `h2` and `verbose`, the only other arg "common" with 1 other
|
2021-03-10 15:37:09 +00:00
|
|
|
// function in the same path is `wid`, so those 3 could be in a struct, but there's no real use
|
2020-10-14 09:33:45 +00:00
|
|
|
// since this is a private method.
|
|
|
|
#[allow(clippy::too_many_arguments)]
|
2019-10-12 11:53:11 +00:00
|
|
|
fn upload_chunk_info_stream(
|
|
|
|
h2: H2Client,
|
|
|
|
wid: u64,
|
|
|
|
stream: impl Stream<Item = Result<bytes::BytesMut, Error>>,
|
|
|
|
prefix: &str,
|
2021-03-24 16:12:05 +00:00
|
|
|
known_chunks: Arc<Mutex<HashSet<[u8; 32]>>>,
|
2019-10-12 11:53:11 +00:00
|
|
|
crypt_config: Option<Arc<CryptConfig>>,
|
2020-07-08 12:06:50 +00:00
|
|
|
compress: bool,
|
2020-02-24 11:48:40 +00:00
|
|
|
verbose: bool,
|
2021-03-24 16:17:18 +00:00
|
|
|
) -> impl Future<Output = Result<UploadStats, Error>> {
|
2020-07-23 07:45:48 +00:00
|
|
|
let total_chunks = Arc::new(AtomicUsize::new(0));
|
|
|
|
let total_chunks2 = total_chunks.clone();
|
|
|
|
let known_chunk_count = Arc::new(AtomicUsize::new(0));
|
|
|
|
let known_chunk_count2 = known_chunk_count.clone();
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
let stream_len = Arc::new(AtomicUsize::new(0));
|
|
|
|
let stream_len2 = stream_len.clone();
|
2020-07-23 07:45:48 +00:00
|
|
|
let reused_len = Arc::new(AtomicUsize::new(0));
|
|
|
|
let reused_len2 = reused_len.clone();
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
let append_chunk_path = format!("{}_index", prefix);
|
|
|
|
let upload_chunk_path = format!("{}_chunk", prefix);
|
|
|
|
let is_fixed_chunk_size = prefix == "fixed";
|
|
|
|
|
|
|
|
let (upload_queue, upload_result) =
|
2020-10-14 09:33:45 +00:00
|
|
|
Self::append_chunk_queue(h2.clone(), wid, append_chunk_path, verbose);
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
let start_time = std::time::Instant::now();
|
|
|
|
|
|
|
|
let index_csum = Arc::new(Mutex::new(Some(openssl::sha::Sha256::new())));
|
|
|
|
let index_csum_2 = index_csum.clone();
|
|
|
|
|
|
|
|
stream
|
|
|
|
.and_then(move |data| {
|
|
|
|
let chunk_len = data.len();
|
|
|
|
|
2020-07-23 07:45:48 +00:00
|
|
|
total_chunks.fetch_add(1, Ordering::SeqCst);
|
2019-10-12 11:53:11 +00:00
|
|
|
let offset = stream_len.fetch_add(chunk_len, Ordering::SeqCst) as u64;
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
let mut chunk_builder = DataChunkBuilder::new(data.as_ref()).compress(compress);
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
if let Some(ref crypt_config) = crypt_config {
|
2020-07-08 12:06:50 +00:00
|
|
|
chunk_builder = chunk_builder.crypt_config(crypt_config);
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
let mut known_chunks = known_chunks.lock().unwrap();
|
|
|
|
let digest = chunk_builder.digest();
|
|
|
|
|
|
|
|
let mut guard = index_csum.lock().unwrap();
|
|
|
|
let csum = guard.as_mut().unwrap();
|
|
|
|
|
|
|
|
let chunk_end = offset + chunk_len as u64;
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
if !is_fixed_chunk_size {
|
|
|
|
csum.update(&chunk_end.to_le_bytes());
|
|
|
|
}
|
2019-10-12 11:53:11 +00:00
|
|
|
csum.update(digest);
|
|
|
|
|
|
|
|
let chunk_is_known = known_chunks.contains(digest);
|
|
|
|
if chunk_is_known {
|
2020-07-23 07:45:48 +00:00
|
|
|
known_chunk_count.fetch_add(1, Ordering::SeqCst);
|
|
|
|
reused_len.fetch_add(chunk_len, Ordering::SeqCst);
|
2019-10-12 11:53:11 +00:00
|
|
|
future::ok(MergedChunkInfo::Known(vec![(offset, *digest)]))
|
|
|
|
} else {
|
|
|
|
known_chunks.insert(*digest);
|
2021-03-24 16:12:05 +00:00
|
|
|
future::ready(chunk_builder.build().map(move |(chunk, digest)| {
|
|
|
|
MergedChunkInfo::New(ChunkInfo {
|
2019-10-12 11:53:11 +00:00
|
|
|
chunk,
|
|
|
|
digest,
|
|
|
|
chunk_len: chunk_len as u64,
|
|
|
|
offset,
|
2021-03-24 16:12:05 +00:00
|
|
|
})
|
|
|
|
}))
|
2019-10-12 11:53:11 +00:00
|
|
|
}
|
|
|
|
})
|
|
|
|
.merge_known_chunks()
|
|
|
|
.try_for_each(move |merged_chunk_info| {
|
2020-12-04 10:53:34 +00:00
|
|
|
let upload_queue = upload_queue.clone();
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
if let MergedChunkInfo::New(chunk_info) = merged_chunk_info {
|
|
|
|
let offset = chunk_info.offset;
|
|
|
|
let digest = chunk_info.digest;
|
|
|
|
let digest_str = digest_to_hex(&digest);
|
|
|
|
|
2020-10-14 09:33:45 +00:00
|
|
|
/* too verbose, needs finer verbosity setting granularity
|
|
|
|
if verbose {
|
2020-02-24 11:48:40 +00:00
|
|
|
println!("upload new chunk {} ({} bytes, offset {})", digest_str,
|
|
|
|
chunk_info.chunk_len, offset);
|
|
|
|
}
|
2020-10-14 09:33:45 +00:00
|
|
|
*/
|
2019-10-12 11:53:11 +00:00
|
|
|
|
2019-12-12 14:27:07 +00:00
|
|
|
let chunk_data = chunk_info.chunk.into_inner();
|
2019-10-12 11:53:11 +00:00
|
|
|
let param = json!({
|
|
|
|
"wid": wid,
|
|
|
|
"digest": digest_str,
|
|
|
|
"size": chunk_info.chunk_len,
|
|
|
|
"encoded-size": chunk_data.len(),
|
|
|
|
});
|
|
|
|
|
|
|
|
let ct = "application/octet-stream";
|
2021-03-24 16:12:05 +00:00
|
|
|
let request = H2Client::request_builder(
|
|
|
|
"localhost",
|
|
|
|
"POST",
|
|
|
|
&upload_chunk_path,
|
|
|
|
Some(param),
|
|
|
|
Some(ct),
|
|
|
|
)
|
|
|
|
.unwrap();
|
2019-10-12 11:53:11 +00:00
|
|
|
let upload_data = Some(bytes::Bytes::from(chunk_data));
|
|
|
|
|
|
|
|
let new_info = MergedChunkInfo::Known(vec![(offset, digest)]);
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
future::Either::Left(h2.send_request(request, upload_data).and_then(
|
|
|
|
move |response| async move {
|
2019-10-12 11:53:11 +00:00
|
|
|
upload_queue
|
|
|
|
.send((new_info, Some(response)))
|
|
|
|
.await
|
2021-03-24 16:12:05 +00:00
|
|
|
.map_err(|err| {
|
|
|
|
format_err!("failed to send to upload queue: {}", err)
|
|
|
|
})
|
|
|
|
},
|
|
|
|
))
|
2019-10-12 11:53:11 +00:00
|
|
|
} else {
|
|
|
|
future::Either::Right(async move {
|
|
|
|
upload_queue
|
|
|
|
.send((merged_chunk_info, None))
|
|
|
|
.await
|
2019-12-12 14:27:07 +00:00
|
|
|
.map_err(|err| format_err!("failed to send to upload queue: {}", err))
|
2019-10-12 11:53:11 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
})
|
2021-03-24 16:12:05 +00:00
|
|
|
.then(move |result| async move { upload_result.await?.and(result) }.boxed())
|
2019-10-12 11:53:11 +00:00
|
|
|
.and_then(move |_| {
|
2020-07-23 07:45:48 +00:00
|
|
|
let duration = start_time.elapsed();
|
2021-03-24 16:17:18 +00:00
|
|
|
let chunk_count = total_chunks2.load(Ordering::SeqCst);
|
|
|
|
let chunk_reused = known_chunk_count2.load(Ordering::SeqCst);
|
|
|
|
let size = stream_len2.load(Ordering::SeqCst);
|
|
|
|
let size_reused = reused_len2.load(Ordering::SeqCst);
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
let mut guard = index_csum_2.lock().unwrap();
|
|
|
|
let csum = guard.take().unwrap().finish();
|
|
|
|
|
2021-03-24 16:17:18 +00:00
|
|
|
futures::future::ok(UploadStats {
|
|
|
|
chunk_count,
|
|
|
|
chunk_reused,
|
|
|
|
size,
|
|
|
|
size_reused,
|
2021-03-24 16:12:05 +00:00
|
|
|
duration,
|
|
|
|
csum,
|
2021-03-24 16:17:18 +00:00
|
|
|
})
|
2019-10-12 11:53:11 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2020-08-25 16:52:31 +00:00
|
|
|
/// Upload speed test - prints result to stderr
|
2020-07-10 07:13:08 +00:00
|
|
|
pub async fn upload_speedtest(&self, verbose: bool) -> Result<f64, Error> {
|
2019-10-12 11:53:11 +00:00
|
|
|
let mut data = vec![];
|
|
|
|
// generate pseudo random byte sequence
|
2021-03-24 16:12:05 +00:00
|
|
|
for i in 0..1024 * 1024 {
|
2019-10-12 11:53:11 +00:00
|
|
|
for j in 0..4 {
|
2021-03-24 16:12:05 +00:00
|
|
|
let byte = ((i >> (j << 3)) & 0xff) as u8;
|
2019-10-12 11:53:11 +00:00
|
|
|
data.push(byte);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
let item_len = data.len();
|
|
|
|
|
|
|
|
let mut repeat = 0;
|
|
|
|
|
2020-07-09 14:16:39 +00:00
|
|
|
let (upload_queue, upload_result) = Self::response_queue(verbose);
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
let start_time = std::time::Instant::now();
|
|
|
|
|
|
|
|
loop {
|
|
|
|
repeat += 1;
|
|
|
|
if start_time.elapsed().as_secs() >= 5 {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
if verbose {
|
|
|
|
eprintln!("send test data ({} bytes)", data.len());
|
|
|
|
}
|
|
|
|
let request =
|
|
|
|
H2Client::request_builder("localhost", "POST", "speedtest", None, None).unwrap();
|
|
|
|
let request_future = self
|
|
|
|
.h2
|
|
|
|
.send_request(request, Some(bytes::Bytes::from(data.clone())))
|
|
|
|
.await?;
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
upload_queue.send(request_future).await?;
|
|
|
|
}
|
|
|
|
|
|
|
|
drop(upload_queue); // close queue
|
|
|
|
|
|
|
|
let _ = upload_result.await?;
|
|
|
|
|
2021-03-24 16:12:05 +00:00
|
|
|
eprintln!(
|
|
|
|
"Uploaded {} chunks in {} seconds.",
|
|
|
|
repeat,
|
|
|
|
start_time.elapsed().as_secs()
|
|
|
|
);
|
|
|
|
let speed = ((item_len * (repeat as usize)) as f64) / start_time.elapsed().as_secs_f64();
|
|
|
|
eprintln!(
|
|
|
|
"Time per request: {} microseconds.",
|
|
|
|
(start_time.elapsed().as_micros()) / (repeat as u128)
|
|
|
|
);
|
2019-10-12 11:53:11 +00:00
|
|
|
|
|
|
|
Ok(speed)
|
|
|
|
}
|
|
|
|
}
|