mirror of
https://github.com/neondatabase/neon.git
synced 2025-12-23 06:09:59 +00:00
Azure blob storage support (#5546)
Adds prototype-level support for [Azure blob storage](https://azure.microsoft.com/en-us/products/storage/blobs). Some corners were cut, see the TODOs and the followup issue #5567 for details. Steps to try it out: * Create a storage account with block blobs (this is a per-storage account setting). * Create a container inside that storage account. * Set the appropriate env vars: `AZURE_STORAGE_ACCOUNT, AZURE_STORAGE_ACCESS_KEY, REMOTE_STORAGE_AZURE_CONTAINER, REMOTE_STORAGE_AZURE_REGION` * Set the env var `ENABLE_REAL_AZURE_REMOTE_STORAGE=y` and run `cargo test -p remote_storage azure` Fixes #5562
This commit is contained in:
946
Cargo.lock
generated
946
Cargo.lock
generated
File diff suppressed because it is too large
Load Diff
@@ -36,6 +36,9 @@ license = "Apache-2.0"
|
||||
[workspace.dependencies]
|
||||
anyhow = { version = "1.0", features = ["backtrace"] }
|
||||
async-compression = { version = "0.4.0", features = ["tokio", "gzip"] }
|
||||
azure_core = "0.16.0"
|
||||
azure_storage = "0.16"
|
||||
azure_storage_blobs = "0.16.0"
|
||||
flate2 = "1.0.26"
|
||||
async-stream = "0.3"
|
||||
async-trait = "0.1"
|
||||
@@ -76,6 +79,7 @@ hex = "0.4"
|
||||
hex-literal = "0.4"
|
||||
hmac = "0.12.1"
|
||||
hostname = "0.3.1"
|
||||
http-types = "2"
|
||||
humantime = "2.1"
|
||||
humantime-serde = "1.1.1"
|
||||
hyper = "0.14"
|
||||
|
||||
@@ -86,7 +86,7 @@ where
|
||||
.stdout(process_log_file)
|
||||
.stderr(same_file_for_stderr)
|
||||
.args(args);
|
||||
let filled_cmd = fill_aws_secrets_vars(fill_rust_env_vars(background_command));
|
||||
let filled_cmd = fill_remote_storage_secrets_vars(fill_rust_env_vars(background_command));
|
||||
filled_cmd.envs(envs);
|
||||
|
||||
let pid_file_to_check = match initial_pid_file {
|
||||
@@ -238,11 +238,13 @@ fn fill_rust_env_vars(cmd: &mut Command) -> &mut Command {
|
||||
filled_cmd
|
||||
}
|
||||
|
||||
fn fill_aws_secrets_vars(mut cmd: &mut Command) -> &mut Command {
|
||||
fn fill_remote_storage_secrets_vars(mut cmd: &mut Command) -> &mut Command {
|
||||
for env_key in [
|
||||
"AWS_ACCESS_KEY_ID",
|
||||
"AWS_SECRET_ACCESS_KEY",
|
||||
"AWS_SESSION_TOKEN",
|
||||
"AZURE_STORAGE_ACCOUNT",
|
||||
"AZURE_STORAGE_ACCESS_KEY",
|
||||
] {
|
||||
if let Ok(value) = std::env::var(env_key) {
|
||||
cmd = cmd.env(env_key, value);
|
||||
|
||||
@@ -96,6 +96,16 @@ prefix_in_bucket = '/test_prefix/'
|
||||
|
||||
`AWS_SECRET_ACCESS_KEY` and `AWS_ACCESS_KEY_ID` env variables can be used to specify the S3 credentials if needed.
|
||||
|
||||
or
|
||||
|
||||
```toml
|
||||
[remote_storage]
|
||||
container_name = 'some-container-name'
|
||||
container_region = 'us-east'
|
||||
prefix_in_container = '/test-prefix/'
|
||||
```
|
||||
|
||||
`AZURE_STORAGE_ACCOUNT` and `AZURE_STORAGE_ACCESS_KEY` env variables can be used to specify the azure credentials if needed.
|
||||
|
||||
## Repository background tasks
|
||||
|
||||
|
||||
@@ -13,6 +13,7 @@ aws-types.workspace = true
|
||||
aws-config.workspace = true
|
||||
aws-sdk-s3.workspace = true
|
||||
aws-credential-types.workspace = true
|
||||
bytes.workspace = true
|
||||
camino.workspace = true
|
||||
hyper = { workspace = true, features = ["stream"] }
|
||||
serde.workspace = true
|
||||
@@ -26,6 +27,12 @@ metrics.workspace = true
|
||||
utils.workspace = true
|
||||
pin-project-lite.workspace = true
|
||||
workspace_hack.workspace = true
|
||||
azure_core.workspace = true
|
||||
azure_storage.workspace = true
|
||||
azure_storage_blobs.workspace = true
|
||||
futures-util.workspace = true
|
||||
http-types.workspace = true
|
||||
itertools.workspace = true
|
||||
|
||||
[dev-dependencies]
|
||||
camino-tempfile.workspace = true
|
||||
|
||||
381
libs/remote_storage/src/azure_blob.rs
Normal file
381
libs/remote_storage/src/azure_blob.rs
Normal file
@@ -0,0 +1,381 @@
|
||||
//! Azure Blob Storage wrapper
|
||||
|
||||
use std::num::NonZeroU32;
|
||||
use std::{borrow::Cow, collections::HashMap, io::Cursor};
|
||||
|
||||
use super::REMOTE_STORAGE_PREFIX_SEPARATOR;
|
||||
use anyhow::Result;
|
||||
use azure_core::request_options::{MaxResults, Metadata, Range};
|
||||
use azure_core::Header;
|
||||
use azure_storage::StorageCredentials;
|
||||
use azure_storage_blobs::prelude::ClientBuilder;
|
||||
use azure_storage_blobs::{
|
||||
blob::operations::GetBlobBuilder,
|
||||
prelude::{BlobClient, ContainerClient},
|
||||
};
|
||||
use futures_util::StreamExt;
|
||||
use http_types::StatusCode;
|
||||
use tokio::io::AsyncRead;
|
||||
use tracing::debug;
|
||||
|
||||
use crate::s3_bucket::RequestKind;
|
||||
use crate::{
|
||||
AzureConfig, ConcurrencyLimiter, Download, DownloadError, RemotePath, RemoteStorage,
|
||||
StorageMetadata,
|
||||
};
|
||||
|
||||
pub struct AzureBlobStorage {
|
||||
client: ContainerClient,
|
||||
prefix_in_container: Option<String>,
|
||||
max_keys_per_list_response: Option<NonZeroU32>,
|
||||
concurrency_limiter: ConcurrencyLimiter,
|
||||
}
|
||||
|
||||
impl AzureBlobStorage {
|
||||
pub fn new(azure_config: &AzureConfig) -> Result<Self> {
|
||||
debug!(
|
||||
"Creating azure remote storage for azure container {}",
|
||||
azure_config.container_name
|
||||
);
|
||||
|
||||
let account =
|
||||
std::env::var("AZURE_STORAGE_ACCOUNT").expect("missing AZURE_STORAGE_ACCOUNT");
|
||||
let access_key =
|
||||
std::env::var("AZURE_STORAGE_ACCESS_KEY").expect("missing AZURE_STORAGE_ACCESS_KEY");
|
||||
|
||||
let credentials = StorageCredentials::access_key(account.clone(), access_key);
|
||||
|
||||
let builder = ClientBuilder::new(account, credentials);
|
||||
|
||||
let client = builder.container_client(azure_config.container_name.to_owned());
|
||||
|
||||
let max_keys_per_list_response =
|
||||
if let Some(limit) = azure_config.max_keys_per_list_response {
|
||||
Some(
|
||||
NonZeroU32::new(limit as u32)
|
||||
.ok_or_else(|| anyhow::anyhow!("max_keys_per_list_response can't be 0"))?,
|
||||
)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
Ok(AzureBlobStorage {
|
||||
client,
|
||||
prefix_in_container: azure_config.prefix_in_container.to_owned(),
|
||||
max_keys_per_list_response,
|
||||
concurrency_limiter: ConcurrencyLimiter::new(azure_config.concurrency_limit.get()),
|
||||
})
|
||||
}
|
||||
|
||||
pub fn relative_path_to_name(&self, path: &RemotePath) -> String {
|
||||
assert_eq!(std::path::MAIN_SEPARATOR, REMOTE_STORAGE_PREFIX_SEPARATOR);
|
||||
let path_string = path
|
||||
.get_path()
|
||||
.as_str()
|
||||
.trim_end_matches(REMOTE_STORAGE_PREFIX_SEPARATOR);
|
||||
match &self.prefix_in_container {
|
||||
Some(prefix) => {
|
||||
if prefix.ends_with(REMOTE_STORAGE_PREFIX_SEPARATOR) {
|
||||
prefix.clone() + path_string
|
||||
} else {
|
||||
format!("{prefix}{REMOTE_STORAGE_PREFIX_SEPARATOR}{path_string}")
|
||||
}
|
||||
}
|
||||
None => path_string.to_string(),
|
||||
}
|
||||
}
|
||||
|
||||
fn name_to_relative_path(&self, key: &str) -> RemotePath {
|
||||
let relative_path =
|
||||
match key.strip_prefix(self.prefix_in_container.as_deref().unwrap_or_default()) {
|
||||
Some(stripped) => stripped,
|
||||
// we rely on Azure to return properly prefixed paths
|
||||
// for requests with a certain prefix
|
||||
None => panic!(
|
||||
"Key {key} does not start with container prefix {:?}",
|
||||
self.prefix_in_container
|
||||
),
|
||||
};
|
||||
RemotePath(
|
||||
relative_path
|
||||
.split(REMOTE_STORAGE_PREFIX_SEPARATOR)
|
||||
.collect(),
|
||||
)
|
||||
}
|
||||
|
||||
async fn download_for_builder(
|
||||
&self,
|
||||
metadata: StorageMetadata,
|
||||
builder: GetBlobBuilder,
|
||||
) -> Result<Download, DownloadError> {
|
||||
let mut response = builder.into_stream();
|
||||
|
||||
// TODO give proper streaming response instead of buffering into RAM
|
||||
// https://github.com/neondatabase/neon/issues/5563
|
||||
let mut buf = Vec::new();
|
||||
while let Some(part) = response.next().await {
|
||||
let part = match part {
|
||||
Ok(l) => l,
|
||||
Err(e) => {
|
||||
return Err(if let Some(http_err) = e.as_http_error() {
|
||||
match http_err.status() {
|
||||
StatusCode::NotFound => DownloadError::NotFound,
|
||||
StatusCode::BadRequest => {
|
||||
DownloadError::BadInput(anyhow::Error::new(e))
|
||||
}
|
||||
_ => DownloadError::Other(anyhow::Error::new(e)),
|
||||
}
|
||||
} else {
|
||||
DownloadError::Other(e.into())
|
||||
});
|
||||
}
|
||||
};
|
||||
let data = part
|
||||
.data
|
||||
.collect()
|
||||
.await
|
||||
.map_err(|e| DownloadError::Other(e.into()))?;
|
||||
buf.extend_from_slice(&data.slice(..));
|
||||
}
|
||||
Ok(Download {
|
||||
download_stream: Box::pin(Cursor::new(buf)),
|
||||
metadata: Some(metadata),
|
||||
})
|
||||
}
|
||||
// TODO get rid of this function once we have metadata included in the response
|
||||
// https://github.com/Azure/azure-sdk-for-rust/issues/1439
|
||||
async fn get_metadata(
|
||||
&self,
|
||||
blob_client: &BlobClient,
|
||||
) -> Result<StorageMetadata, DownloadError> {
|
||||
let builder = blob_client.get_metadata();
|
||||
|
||||
match builder.into_future().await {
|
||||
Ok(r) => {
|
||||
let mut map = HashMap::new();
|
||||
|
||||
for md in r.metadata.iter() {
|
||||
map.insert(
|
||||
md.name().as_str().to_string(),
|
||||
md.value().as_str().to_string(),
|
||||
);
|
||||
}
|
||||
Ok(StorageMetadata(map))
|
||||
}
|
||||
Err(e) => {
|
||||
return Err(if let Some(http_err) = e.as_http_error() {
|
||||
match http_err.status() {
|
||||
StatusCode::NotFound => DownloadError::NotFound,
|
||||
StatusCode::BadRequest => DownloadError::BadInput(anyhow::Error::new(e)),
|
||||
_ => DownloadError::Other(anyhow::Error::new(e)),
|
||||
}
|
||||
} else {
|
||||
DownloadError::Other(e.into())
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn permit(&self, kind: RequestKind) -> tokio::sync::SemaphorePermit<'_> {
|
||||
self.concurrency_limiter
|
||||
.acquire(kind)
|
||||
.await
|
||||
.expect("semaphore is never closed")
|
||||
}
|
||||
}
|
||||
|
||||
fn to_azure_metadata(metadata: StorageMetadata) -> Metadata {
|
||||
let mut res = Metadata::new();
|
||||
for (k, v) in metadata.0.into_iter() {
|
||||
res.insert(k, v);
|
||||
}
|
||||
res
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl RemoteStorage for AzureBlobStorage {
|
||||
async fn list_prefixes(
|
||||
&self,
|
||||
prefix: Option<&RemotePath>,
|
||||
) -> Result<Vec<RemotePath>, DownloadError> {
|
||||
// get the passed prefix or if it is not set use prefix_in_bucket value
|
||||
let list_prefix = prefix
|
||||
.map(|p| self.relative_path_to_name(p))
|
||||
.or_else(|| self.prefix_in_container.clone())
|
||||
.map(|mut p| {
|
||||
// required to end with a separator
|
||||
// otherwise request will return only the entry of a prefix
|
||||
if !p.ends_with(REMOTE_STORAGE_PREFIX_SEPARATOR) {
|
||||
p.push(REMOTE_STORAGE_PREFIX_SEPARATOR);
|
||||
}
|
||||
p
|
||||
});
|
||||
|
||||
let mut builder = self
|
||||
.client
|
||||
.list_blobs()
|
||||
.delimiter(REMOTE_STORAGE_PREFIX_SEPARATOR.to_string());
|
||||
|
||||
if let Some(prefix) = list_prefix {
|
||||
builder = builder.prefix(Cow::from(prefix.to_owned()));
|
||||
}
|
||||
|
||||
if let Some(limit) = self.max_keys_per_list_response {
|
||||
builder = builder.max_results(MaxResults::new(limit));
|
||||
}
|
||||
|
||||
let mut response = builder.into_stream();
|
||||
let mut res = Vec::new();
|
||||
while let Some(l) = response.next().await {
|
||||
let entry = match l {
|
||||
Ok(l) => l,
|
||||
Err(e) => {
|
||||
return Err(if let Some(http_err) = e.as_http_error() {
|
||||
match http_err.status() {
|
||||
StatusCode::NotFound => DownloadError::NotFound,
|
||||
StatusCode::BadRequest => {
|
||||
DownloadError::BadInput(anyhow::Error::new(e))
|
||||
}
|
||||
_ => DownloadError::Other(anyhow::Error::new(e)),
|
||||
}
|
||||
} else {
|
||||
DownloadError::Other(e.into())
|
||||
});
|
||||
}
|
||||
};
|
||||
let name_iter = entry
|
||||
.blobs
|
||||
.prefixes()
|
||||
.map(|prefix| self.name_to_relative_path(&prefix.name));
|
||||
res.extend(name_iter);
|
||||
}
|
||||
Ok(res)
|
||||
}
|
||||
|
||||
async fn list_files(&self, folder: Option<&RemotePath>) -> anyhow::Result<Vec<RemotePath>> {
|
||||
let folder_name = folder
|
||||
.map(|p| self.relative_path_to_name(p))
|
||||
.or_else(|| self.prefix_in_container.clone());
|
||||
|
||||
let mut builder = self.client.list_blobs();
|
||||
|
||||
if let Some(folder_name) = folder_name {
|
||||
builder = builder.prefix(Cow::from(folder_name.to_owned()));
|
||||
}
|
||||
|
||||
if let Some(limit) = self.max_keys_per_list_response {
|
||||
builder = builder.max_results(MaxResults::new(limit));
|
||||
}
|
||||
|
||||
let mut response = builder.into_stream();
|
||||
let mut res = Vec::new();
|
||||
while let Some(l) = response.next().await {
|
||||
let entry = l.map_err(anyhow::Error::new)?;
|
||||
let name_iter = entry
|
||||
.blobs
|
||||
.blobs()
|
||||
.map(|bl| self.name_to_relative_path(&bl.name));
|
||||
res.extend(name_iter);
|
||||
}
|
||||
Ok(res)
|
||||
}
|
||||
|
||||
async fn upload(
|
||||
&self,
|
||||
mut from: impl AsyncRead + Unpin + Send + Sync + 'static,
|
||||
data_size_bytes: usize,
|
||||
to: &RemotePath,
|
||||
metadata: Option<StorageMetadata>,
|
||||
) -> anyhow::Result<()> {
|
||||
let _permit = self.permit(RequestKind::Put).await;
|
||||
let blob_client = self.client.blob_client(self.relative_path_to_name(to));
|
||||
|
||||
// TODO FIX THIS UGLY HACK and don't buffer the entire object
|
||||
// into RAM here, but use the streaming interface. For that,
|
||||
// we'd have to change the interface though...
|
||||
// https://github.com/neondatabase/neon/issues/5563
|
||||
let mut buf = Vec::with_capacity(data_size_bytes);
|
||||
tokio::io::copy(&mut from, &mut buf).await?;
|
||||
let body = azure_core::Body::Bytes(buf.into());
|
||||
|
||||
let mut builder = blob_client.put_block_blob(body);
|
||||
|
||||
if let Some(metadata) = metadata {
|
||||
builder = builder.metadata(to_azure_metadata(metadata));
|
||||
}
|
||||
|
||||
let _response = builder.into_future().await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn download(&self, from: &RemotePath) -> Result<Download, DownloadError> {
|
||||
let _permit = self.permit(RequestKind::Get).await;
|
||||
let blob_client = self.client.blob_client(self.relative_path_to_name(from));
|
||||
|
||||
let metadata = self.get_metadata(&blob_client).await?;
|
||||
|
||||
let builder = blob_client.get();
|
||||
|
||||
self.download_for_builder(metadata, builder).await
|
||||
}
|
||||
|
||||
async fn download_byte_range(
|
||||
&self,
|
||||
from: &RemotePath,
|
||||
start_inclusive: u64,
|
||||
end_exclusive: Option<u64>,
|
||||
) -> Result<Download, DownloadError> {
|
||||
let _permit = self.permit(RequestKind::Get).await;
|
||||
let blob_client = self.client.blob_client(self.relative_path_to_name(from));
|
||||
|
||||
let metadata = self.get_metadata(&blob_client).await?;
|
||||
|
||||
let mut builder = blob_client.get();
|
||||
|
||||
if let Some(end_exclusive) = end_exclusive {
|
||||
builder = builder.range(Range::new(start_inclusive, end_exclusive));
|
||||
} else {
|
||||
// Open ranges are not supported by the SDK so we work around
|
||||
// by setting the upper limit extremely high (but high enough
|
||||
// to still be representable by signed 64 bit integers).
|
||||
// TODO remove workaround once the SDK adds open range support
|
||||
// https://github.com/Azure/azure-sdk-for-rust/issues/1438
|
||||
let end_exclusive = u64::MAX << 4;
|
||||
builder = builder.range(Range::new(start_inclusive, end_exclusive));
|
||||
}
|
||||
|
||||
self.download_for_builder(metadata, builder).await
|
||||
}
|
||||
|
||||
async fn delete(&self, path: &RemotePath) -> anyhow::Result<()> {
|
||||
let _permit = self.permit(RequestKind::Delete).await;
|
||||
let blob_client = self.client.blob_client(self.relative_path_to_name(path));
|
||||
|
||||
let builder = blob_client.delete();
|
||||
|
||||
match builder.into_future().await {
|
||||
Ok(_response) => Ok(()),
|
||||
Err(e) => {
|
||||
if let Some(http_err) = e.as_http_error() {
|
||||
if http_err.status() == StatusCode::NotFound {
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
Err(anyhow::Error::new(e))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn delete_objects<'a>(&self, paths: &'a [RemotePath]) -> anyhow::Result<()> {
|
||||
// Permit is already obtained by inner delete function
|
||||
|
||||
// TODO batch requests are also not supported by the SDK
|
||||
// https://github.com/Azure/azure-sdk-for-rust/issues/1068
|
||||
// https://github.com/Azure/azure-sdk-for-rust/issues/1249
|
||||
for path in paths {
|
||||
self.delete(path).await?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -4,7 +4,10 @@
|
||||
//! [`RemoteStorage`] trait a CRUD-like generic abstraction to use for adapting external storages with a few implementations:
|
||||
//! * [`local_fs`] allows to use local file system as an external storage
|
||||
//! * [`s3_bucket`] uses AWS S3 bucket as an external storage
|
||||
//! * [`azure_blob`] allows to use Azure Blob storage as an external storage
|
||||
//!
|
||||
|
||||
mod azure_blob;
|
||||
mod local_fs;
|
||||
mod s3_bucket;
|
||||
mod simulate_failures;
|
||||
@@ -21,11 +24,15 @@ use anyhow::{bail, Context};
|
||||
use camino::{Utf8Path, Utf8PathBuf};
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
use tokio::io;
|
||||
use tokio::{io, sync::Semaphore};
|
||||
use toml_edit::Item;
|
||||
use tracing::info;
|
||||
|
||||
pub use self::{local_fs::LocalFs, s3_bucket::S3Bucket, simulate_failures::UnreliableWrapper};
|
||||
pub use self::{
|
||||
azure_blob::AzureBlobStorage, local_fs::LocalFs, s3_bucket::S3Bucket,
|
||||
simulate_failures::UnreliableWrapper,
|
||||
};
|
||||
use s3_bucket::RequestKind;
|
||||
|
||||
/// How many different timelines can be processed simultaneously when synchronizing layers with the remote storage.
|
||||
/// During regular work, pageserver produces one layer file per timeline checkpoint, with bursts of concurrency
|
||||
@@ -39,6 +46,11 @@ pub const DEFAULT_REMOTE_STORAGE_MAX_SYNC_ERRORS: u32 = 10;
|
||||
/// ~3500 PUT/COPY/POST/DELETE or 5500 GET/HEAD S3 requests
|
||||
/// <https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/>
|
||||
pub const DEFAULT_REMOTE_STORAGE_S3_CONCURRENCY_LIMIT: usize = 100;
|
||||
/// We set this a little bit low as we currently buffer the entire file into RAM
|
||||
///
|
||||
/// Here, a limit of max 20k concurrent connections was noted.
|
||||
/// <https://learn.microsoft.com/en-us/answers/questions/1301863/is-there-any-limitation-to-concurrent-connections>
|
||||
pub const DEFAULT_REMOTE_STORAGE_AZURE_CONCURRENCY_LIMIT: usize = 30;
|
||||
/// No limits on the client side, which currenltly means 1000 for AWS S3.
|
||||
/// <https://docs.aws.amazon.com/AmazonS3/latest/API/API_ListObjectsV2.html#API_ListObjectsV2_RequestSyntax>
|
||||
pub const DEFAULT_MAX_KEYS_PER_LIST_RESPONSE: Option<i32> = None;
|
||||
@@ -217,6 +229,7 @@ impl std::error::Error for DownloadError {}
|
||||
pub enum GenericRemoteStorage {
|
||||
LocalFs(LocalFs),
|
||||
AwsS3(Arc<S3Bucket>),
|
||||
AzureBlob(Arc<AzureBlobStorage>),
|
||||
Unreliable(Arc<UnreliableWrapper>),
|
||||
}
|
||||
|
||||
@@ -228,6 +241,7 @@ impl GenericRemoteStorage {
|
||||
match self {
|
||||
Self::LocalFs(s) => s.list_files(folder).await,
|
||||
Self::AwsS3(s) => s.list_files(folder).await,
|
||||
Self::AzureBlob(s) => s.list_files(folder).await,
|
||||
Self::Unreliable(s) => s.list_files(folder).await,
|
||||
}
|
||||
}
|
||||
@@ -242,6 +256,7 @@ impl GenericRemoteStorage {
|
||||
match self {
|
||||
Self::LocalFs(s) => s.list_prefixes(prefix).await,
|
||||
Self::AwsS3(s) => s.list_prefixes(prefix).await,
|
||||
Self::AzureBlob(s) => s.list_prefixes(prefix).await,
|
||||
Self::Unreliable(s) => s.list_prefixes(prefix).await,
|
||||
}
|
||||
}
|
||||
@@ -256,6 +271,7 @@ impl GenericRemoteStorage {
|
||||
match self {
|
||||
Self::LocalFs(s) => s.upload(from, data_size_bytes, to, metadata).await,
|
||||
Self::AwsS3(s) => s.upload(from, data_size_bytes, to, metadata).await,
|
||||
Self::AzureBlob(s) => s.upload(from, data_size_bytes, to, metadata).await,
|
||||
Self::Unreliable(s) => s.upload(from, data_size_bytes, to, metadata).await,
|
||||
}
|
||||
}
|
||||
@@ -264,6 +280,7 @@ impl GenericRemoteStorage {
|
||||
match self {
|
||||
Self::LocalFs(s) => s.download(from).await,
|
||||
Self::AwsS3(s) => s.download(from).await,
|
||||
Self::AzureBlob(s) => s.download(from).await,
|
||||
Self::Unreliable(s) => s.download(from).await,
|
||||
}
|
||||
}
|
||||
@@ -283,6 +300,10 @@ impl GenericRemoteStorage {
|
||||
s.download_byte_range(from, start_inclusive, end_exclusive)
|
||||
.await
|
||||
}
|
||||
Self::AzureBlob(s) => {
|
||||
s.download_byte_range(from, start_inclusive, end_exclusive)
|
||||
.await
|
||||
}
|
||||
Self::Unreliable(s) => {
|
||||
s.download_byte_range(from, start_inclusive, end_exclusive)
|
||||
.await
|
||||
@@ -294,6 +315,7 @@ impl GenericRemoteStorage {
|
||||
match self {
|
||||
Self::LocalFs(s) => s.delete(path).await,
|
||||
Self::AwsS3(s) => s.delete(path).await,
|
||||
Self::AzureBlob(s) => s.delete(path).await,
|
||||
Self::Unreliable(s) => s.delete(path).await,
|
||||
}
|
||||
}
|
||||
@@ -302,6 +324,7 @@ impl GenericRemoteStorage {
|
||||
match self {
|
||||
Self::LocalFs(s) => s.delete_objects(paths).await,
|
||||
Self::AwsS3(s) => s.delete_objects(paths).await,
|
||||
Self::AzureBlob(s) => s.delete_objects(paths).await,
|
||||
Self::Unreliable(s) => s.delete_objects(paths).await,
|
||||
}
|
||||
}
|
||||
@@ -319,6 +342,11 @@ impl GenericRemoteStorage {
|
||||
s3_config.bucket_name, s3_config.bucket_region, s3_config.prefix_in_bucket, s3_config.endpoint);
|
||||
Self::AwsS3(Arc::new(S3Bucket::new(s3_config)?))
|
||||
}
|
||||
RemoteStorageKind::AzureContainer(azure_config) => {
|
||||
info!("Using azure container '{}' in region '{}' as a remote storage, prefix in container: '{:?}'",
|
||||
azure_config.container_name, azure_config.container_region, azure_config.prefix_in_container);
|
||||
Self::AzureBlob(Arc::new(AzureBlobStorage::new(azure_config)?))
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@@ -383,6 +411,9 @@ pub enum RemoteStorageKind {
|
||||
/// AWS S3 based storage, storing all files in the S3 bucket
|
||||
/// specified by the config
|
||||
AwsS3(S3Config),
|
||||
/// Azure Blob based storage, storing all files in the container
|
||||
/// specified by the config
|
||||
AzureContainer(AzureConfig),
|
||||
}
|
||||
|
||||
/// AWS S3 bucket coordinates and access credentials to manage the bucket contents (read and write).
|
||||
@@ -422,11 +453,45 @@ impl Debug for S3Config {
|
||||
}
|
||||
}
|
||||
|
||||
/// Azure bucket coordinates and access credentials to manage the bucket contents (read and write).
|
||||
#[derive(Clone, PartialEq, Eq)]
|
||||
pub struct AzureConfig {
|
||||
/// Name of the container to connect to.
|
||||
pub container_name: String,
|
||||
/// The region where the bucket is located at.
|
||||
pub container_region: String,
|
||||
/// A "subfolder" in the container, to use the same container separately by multiple remote storage users at once.
|
||||
pub prefix_in_container: Option<String>,
|
||||
/// Azure has various limits on its API calls, we need not to exceed those.
|
||||
/// See [`DEFAULT_REMOTE_STORAGE_AZURE_CONCURRENCY_LIMIT`] for more details.
|
||||
pub concurrency_limit: NonZeroUsize,
|
||||
pub max_keys_per_list_response: Option<i32>,
|
||||
}
|
||||
|
||||
impl Debug for AzureConfig {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("AzureConfig")
|
||||
.field("bucket_name", &self.container_name)
|
||||
.field("bucket_region", &self.container_region)
|
||||
.field("prefix_in_bucket", &self.prefix_in_container)
|
||||
.field("concurrency_limit", &self.concurrency_limit)
|
||||
.field(
|
||||
"max_keys_per_list_response",
|
||||
&self.max_keys_per_list_response,
|
||||
)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl RemoteStorageConfig {
|
||||
pub fn from_toml(toml: &toml_edit::Item) -> anyhow::Result<Option<RemoteStorageConfig>> {
|
||||
let local_path = toml.get("local_path");
|
||||
let bucket_name = toml.get("bucket_name");
|
||||
let bucket_region = toml.get("bucket_region");
|
||||
let container_name = toml.get("container_name");
|
||||
let container_region = toml.get("container_region");
|
||||
|
||||
let use_azure = container_name.is_some() && container_region.is_some();
|
||||
|
||||
let max_concurrent_syncs = NonZeroUsize::new(
|
||||
parse_optional_integer("max_concurrent_syncs", toml)?
|
||||
@@ -440,9 +505,13 @@ impl RemoteStorageConfig {
|
||||
)
|
||||
.context("Failed to parse 'max_sync_errors' as a positive integer")?;
|
||||
|
||||
let default_concurrency_limit = if use_azure {
|
||||
DEFAULT_REMOTE_STORAGE_AZURE_CONCURRENCY_LIMIT
|
||||
} else {
|
||||
DEFAULT_REMOTE_STORAGE_S3_CONCURRENCY_LIMIT
|
||||
};
|
||||
let concurrency_limit = NonZeroUsize::new(
|
||||
parse_optional_integer("concurrency_limit", toml)?
|
||||
.unwrap_or(DEFAULT_REMOTE_STORAGE_S3_CONCURRENCY_LIMIT),
|
||||
parse_optional_integer("concurrency_limit", toml)?.unwrap_or(default_concurrency_limit),
|
||||
)
|
||||
.context("Failed to parse 'concurrency_limit' as a positive integer")?;
|
||||
|
||||
@@ -451,33 +520,70 @@ impl RemoteStorageConfig {
|
||||
.context("Failed to parse 'max_keys_per_list_response' as a positive integer")?
|
||||
.or(DEFAULT_MAX_KEYS_PER_LIST_RESPONSE);
|
||||
|
||||
let storage = match (local_path, bucket_name, bucket_region) {
|
||||
let endpoint = toml
|
||||
.get("endpoint")
|
||||
.map(|endpoint| parse_toml_string("endpoint", endpoint))
|
||||
.transpose()?;
|
||||
|
||||
let storage = match (
|
||||
local_path,
|
||||
bucket_name,
|
||||
bucket_region,
|
||||
container_name,
|
||||
container_region,
|
||||
) {
|
||||
// no 'local_path' nor 'bucket_name' options are provided, consider this remote storage disabled
|
||||
(None, None, None) => return Ok(None),
|
||||
(_, Some(_), None) => {
|
||||
(None, None, None, None, None) => return Ok(None),
|
||||
(_, Some(_), None, ..) => {
|
||||
bail!("'bucket_region' option is mandatory if 'bucket_name' is given ")
|
||||
}
|
||||
(_, None, Some(_)) => {
|
||||
(_, None, Some(_), ..) => {
|
||||
bail!("'bucket_name' option is mandatory if 'bucket_region' is given ")
|
||||
}
|
||||
(None, Some(bucket_name), Some(bucket_region)) => RemoteStorageKind::AwsS3(S3Config {
|
||||
bucket_name: parse_toml_string("bucket_name", bucket_name)?,
|
||||
bucket_region: parse_toml_string("bucket_region", bucket_region)?,
|
||||
prefix_in_bucket: toml
|
||||
.get("prefix_in_bucket")
|
||||
.map(|prefix_in_bucket| parse_toml_string("prefix_in_bucket", prefix_in_bucket))
|
||||
.transpose()?,
|
||||
endpoint: toml
|
||||
.get("endpoint")
|
||||
.map(|endpoint| parse_toml_string("endpoint", endpoint))
|
||||
.transpose()?,
|
||||
concurrency_limit,
|
||||
max_keys_per_list_response,
|
||||
}),
|
||||
(Some(local_path), None, None) => RemoteStorageKind::LocalFs(Utf8PathBuf::from(
|
||||
parse_toml_string("local_path", local_path)?,
|
||||
)),
|
||||
(Some(_), Some(_), _) => bail!("local_path and bucket_name are mutually exclusive"),
|
||||
(None, Some(bucket_name), Some(bucket_region), ..) => {
|
||||
RemoteStorageKind::AwsS3(S3Config {
|
||||
bucket_name: parse_toml_string("bucket_name", bucket_name)?,
|
||||
bucket_region: parse_toml_string("bucket_region", bucket_region)?,
|
||||
prefix_in_bucket: toml
|
||||
.get("prefix_in_bucket")
|
||||
.map(|prefix_in_bucket| {
|
||||
parse_toml_string("prefix_in_bucket", prefix_in_bucket)
|
||||
})
|
||||
.transpose()?,
|
||||
endpoint,
|
||||
concurrency_limit,
|
||||
max_keys_per_list_response,
|
||||
})
|
||||
}
|
||||
(_, _, _, Some(_), None) => {
|
||||
bail!("'container_name' option is mandatory if 'container_region' is given ")
|
||||
}
|
||||
(_, _, _, None, Some(_)) => {
|
||||
bail!("'container_name' option is mandatory if 'container_region' is given ")
|
||||
}
|
||||
(None, None, None, Some(container_name), Some(container_region)) => {
|
||||
RemoteStorageKind::AzureContainer(AzureConfig {
|
||||
container_name: parse_toml_string("container_name", container_name)?,
|
||||
container_region: parse_toml_string("container_region", container_region)?,
|
||||
prefix_in_container: toml
|
||||
.get("prefix_in_container")
|
||||
.map(|prefix_in_container| {
|
||||
parse_toml_string("prefix_in_container", prefix_in_container)
|
||||
})
|
||||
.transpose()?,
|
||||
concurrency_limit,
|
||||
max_keys_per_list_response,
|
||||
})
|
||||
}
|
||||
(Some(local_path), None, None, None, None) => RemoteStorageKind::LocalFs(
|
||||
Utf8PathBuf::from(parse_toml_string("local_path", local_path)?),
|
||||
),
|
||||
(Some(_), Some(_), ..) => {
|
||||
bail!("'local_path' and 'bucket_name' are mutually exclusive")
|
||||
}
|
||||
(Some(_), _, _, Some(_), Some(_)) => {
|
||||
bail!("local_path and 'container_name' are mutually exclusive")
|
||||
}
|
||||
};
|
||||
|
||||
Ok(Some(RemoteStorageConfig {
|
||||
@@ -513,6 +619,46 @@ fn parse_toml_string(name: &str, item: &Item) -> anyhow::Result<String> {
|
||||
Ok(s.to_string())
|
||||
}
|
||||
|
||||
struct ConcurrencyLimiter {
|
||||
// Every request to S3 can be throttled or cancelled, if a certain number of requests per second is exceeded.
|
||||
// Same goes to IAM, which is queried before every S3 request, if enabled. IAM has even lower RPS threshold.
|
||||
// The helps to ensure we don't exceed the thresholds.
|
||||
write: Arc<Semaphore>,
|
||||
read: Arc<Semaphore>,
|
||||
}
|
||||
|
||||
impl ConcurrencyLimiter {
|
||||
fn for_kind(&self, kind: RequestKind) -> &Arc<Semaphore> {
|
||||
match kind {
|
||||
RequestKind::Get => &self.read,
|
||||
RequestKind::Put => &self.write,
|
||||
RequestKind::List => &self.read,
|
||||
RequestKind::Delete => &self.write,
|
||||
}
|
||||
}
|
||||
|
||||
async fn acquire(
|
||||
&self,
|
||||
kind: RequestKind,
|
||||
) -> Result<tokio::sync::SemaphorePermit<'_>, tokio::sync::AcquireError> {
|
||||
self.for_kind(kind).acquire().await
|
||||
}
|
||||
|
||||
async fn acquire_owned(
|
||||
&self,
|
||||
kind: RequestKind,
|
||||
) -> Result<tokio::sync::OwnedSemaphorePermit, tokio::sync::AcquireError> {
|
||||
Arc::clone(self.for_kind(kind)).acquire_owned().await
|
||||
}
|
||||
|
||||
fn new(limit: usize) -> ConcurrencyLimiter {
|
||||
Self {
|
||||
read: Arc::new(Semaphore::new(limit)),
|
||||
write: Arc::new(Semaphore::new(limit)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
@@ -4,7 +4,7 @@
|
||||
//! allowing multiple api users to independently work with the same S3 bucket, if
|
||||
//! their bucket prefixes are both specified and different.
|
||||
|
||||
use std::{borrow::Cow, sync::Arc};
|
||||
use std::borrow::Cow;
|
||||
|
||||
use anyhow::Context;
|
||||
use aws_config::{
|
||||
@@ -24,22 +24,20 @@ use aws_sdk_s3::{
|
||||
use aws_smithy_http::body::SdkBody;
|
||||
use hyper::Body;
|
||||
use scopeguard::ScopeGuard;
|
||||
use tokio::{
|
||||
io::{self, AsyncRead},
|
||||
sync::Semaphore,
|
||||
};
|
||||
use tokio::io::{self, AsyncRead};
|
||||
use tokio_util::io::ReaderStream;
|
||||
use tracing::debug;
|
||||
|
||||
use super::StorageMetadata;
|
||||
use crate::{
|
||||
Download, DownloadError, RemotePath, RemoteStorage, S3Config, MAX_KEYS_PER_DELETE,
|
||||
REMOTE_STORAGE_PREFIX_SEPARATOR,
|
||||
ConcurrencyLimiter, Download, DownloadError, RemotePath, RemoteStorage, S3Config,
|
||||
MAX_KEYS_PER_DELETE, REMOTE_STORAGE_PREFIX_SEPARATOR,
|
||||
};
|
||||
|
||||
pub(super) mod metrics;
|
||||
|
||||
use self::metrics::{AttemptOutcome, RequestKind};
|
||||
use self::metrics::AttemptOutcome;
|
||||
pub(super) use self::metrics::RequestKind;
|
||||
|
||||
/// AWS S3 storage.
|
||||
pub struct S3Bucket {
|
||||
@@ -50,46 +48,6 @@ pub struct S3Bucket {
|
||||
concurrency_limiter: ConcurrencyLimiter,
|
||||
}
|
||||
|
||||
struct ConcurrencyLimiter {
|
||||
// Every request to S3 can be throttled or cancelled, if a certain number of requests per second is exceeded.
|
||||
// Same goes to IAM, which is queried before every S3 request, if enabled. IAM has even lower RPS threshold.
|
||||
// The helps to ensure we don't exceed the thresholds.
|
||||
write: Arc<Semaphore>,
|
||||
read: Arc<Semaphore>,
|
||||
}
|
||||
|
||||
impl ConcurrencyLimiter {
|
||||
fn for_kind(&self, kind: RequestKind) -> &Arc<Semaphore> {
|
||||
match kind {
|
||||
RequestKind::Get => &self.read,
|
||||
RequestKind::Put => &self.write,
|
||||
RequestKind::List => &self.read,
|
||||
RequestKind::Delete => &self.write,
|
||||
}
|
||||
}
|
||||
|
||||
async fn acquire(
|
||||
&self,
|
||||
kind: RequestKind,
|
||||
) -> Result<tokio::sync::SemaphorePermit<'_>, tokio::sync::AcquireError> {
|
||||
self.for_kind(kind).acquire().await
|
||||
}
|
||||
|
||||
async fn acquire_owned(
|
||||
&self,
|
||||
kind: RequestKind,
|
||||
) -> Result<tokio::sync::OwnedSemaphorePermit, tokio::sync::AcquireError> {
|
||||
Arc::clone(self.for_kind(kind)).acquire_owned().await
|
||||
}
|
||||
|
||||
fn new(limit: usize) -> ConcurrencyLimiter {
|
||||
Self {
|
||||
read: Arc::new(Semaphore::new(limit)),
|
||||
write: Arc::new(Semaphore::new(limit)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
struct GetObjectRequest {
|
||||
bucket: String,
|
||||
|
||||
@@ -6,7 +6,7 @@ use once_cell::sync::Lazy;
|
||||
pub(super) static BUCKET_METRICS: Lazy<BucketMetrics> = Lazy::new(Default::default);
|
||||
|
||||
#[derive(Clone, Copy, Debug)]
|
||||
pub(super) enum RequestKind {
|
||||
pub(crate) enum RequestKind {
|
||||
Get = 0,
|
||||
Put = 1,
|
||||
Delete = 2,
|
||||
|
||||
619
libs/remote_storage/tests/test_real_azure.rs
Normal file
619
libs/remote_storage/tests/test_real_azure.rs
Normal file
@@ -0,0 +1,619 @@
|
||||
use std::collections::HashSet;
|
||||
use std::env;
|
||||
use std::num::{NonZeroU32, NonZeroUsize};
|
||||
use std::ops::ControlFlow;
|
||||
use std::path::PathBuf;
|
||||
use std::sync::Arc;
|
||||
use std::time::UNIX_EPOCH;
|
||||
|
||||
use anyhow::Context;
|
||||
use camino::Utf8Path;
|
||||
use once_cell::sync::OnceCell;
|
||||
use remote_storage::{
|
||||
AzureConfig, Download, GenericRemoteStorage, RemotePath, RemoteStorageConfig, RemoteStorageKind,
|
||||
};
|
||||
use test_context::{test_context, AsyncTestContext};
|
||||
use tokio::task::JoinSet;
|
||||
use tracing::{debug, error, info};
|
||||
|
||||
static LOGGING_DONE: OnceCell<()> = OnceCell::new();
|
||||
|
||||
const ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME: &str = "ENABLE_REAL_AZURE_REMOTE_STORAGE";
|
||||
|
||||
const BASE_PREFIX: &str = "test";
|
||||
|
||||
/// Tests that the Azure client can list all prefixes, even if the response comes paginated and requires multiple HTTP queries.
|
||||
/// Uses real Azure and requires [`ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME`] and related Azure cred env vars specified.
|
||||
/// See the client creation in [`create_azure_client`] for details on the required env vars.
|
||||
/// If real Azure tests are disabled, the test passes, skipping any real test run: currently, there's no way to mark the test ignored in runtime with the
|
||||
/// deafult test framework, see https://github.com/rust-lang/rust/issues/68007 for details.
|
||||
///
|
||||
/// First, the test creates a set of Azure blobs with keys `/${random_prefix_part}/${base_prefix_str}/sub_prefix_${i}/blob_${i}` in [`upload_azure_data`]
|
||||
/// where
|
||||
/// * `random_prefix_part` is set for the entire Azure client during the Azure client creation in [`create_azure_client`], to avoid multiple test runs interference
|
||||
/// * `base_prefix_str` is a common prefix to use in the client requests: we would want to ensure that the client is able to list nested prefixes inside the bucket
|
||||
///
|
||||
/// Then, verifies that the client does return correct prefixes when queried:
|
||||
/// * with no prefix, it lists everything after its `${random_prefix_part}/` — that should be `${base_prefix_str}` value only
|
||||
/// * with `${base_prefix_str}/` prefix, it lists every `sub_prefix_${i}`
|
||||
///
|
||||
/// With the real Azure enabled and `#[cfg(test)]` Rust configuration used, the Azure client test adds a `max-keys` param to limit the response keys.
|
||||
/// This way, we are able to test the pagination implicitly, by ensuring all results are returned from the remote storage and avoid uploading too many blobs to Azure.
|
||||
///
|
||||
/// Lastly, the test attempts to clean up and remove all uploaded Azure files.
|
||||
/// If any errors appear during the clean up, they get logged, but the test is not failed or stopped until clean up is finished.
|
||||
#[test_context(MaybeEnabledAzureWithTestBlobs)]
|
||||
#[tokio::test]
|
||||
async fn azure_pagination_should_work(
|
||||
ctx: &mut MaybeEnabledAzureWithTestBlobs,
|
||||
) -> anyhow::Result<()> {
|
||||
let ctx = match ctx {
|
||||
MaybeEnabledAzureWithTestBlobs::Enabled(ctx) => ctx,
|
||||
MaybeEnabledAzureWithTestBlobs::Disabled => return Ok(()),
|
||||
MaybeEnabledAzureWithTestBlobs::UploadsFailed(e, _) => {
|
||||
anyhow::bail!("Azure init failed: {e:?}")
|
||||
}
|
||||
};
|
||||
|
||||
let test_client = Arc::clone(&ctx.enabled.client);
|
||||
let expected_remote_prefixes = ctx.remote_prefixes.clone();
|
||||
|
||||
let base_prefix = RemotePath::new(Utf8Path::new(ctx.enabled.base_prefix))
|
||||
.context("common_prefix construction")?;
|
||||
let root_remote_prefixes = test_client
|
||||
.list_prefixes(None)
|
||||
.await
|
||||
.context("client list root prefixes failure")?
|
||||
.into_iter()
|
||||
.collect::<HashSet<_>>();
|
||||
assert_eq!(
|
||||
root_remote_prefixes, HashSet::from([base_prefix.clone()]),
|
||||
"remote storage root prefixes list mismatches with the uploads. Returned prefixes: {root_remote_prefixes:?}"
|
||||
);
|
||||
|
||||
let nested_remote_prefixes = test_client
|
||||
.list_prefixes(Some(&base_prefix))
|
||||
.await
|
||||
.context("client list nested prefixes failure")?
|
||||
.into_iter()
|
||||
.collect::<HashSet<_>>();
|
||||
let remote_only_prefixes = nested_remote_prefixes
|
||||
.difference(&expected_remote_prefixes)
|
||||
.collect::<HashSet<_>>();
|
||||
let missing_uploaded_prefixes = expected_remote_prefixes
|
||||
.difference(&nested_remote_prefixes)
|
||||
.collect::<HashSet<_>>();
|
||||
assert_eq!(
|
||||
remote_only_prefixes.len() + missing_uploaded_prefixes.len(), 0,
|
||||
"remote storage nested prefixes list mismatches with the uploads. Remote only prefixes: {remote_only_prefixes:?}, missing uploaded prefixes: {missing_uploaded_prefixes:?}",
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Tests that Azure client can list all files in a folder, even if the response comes paginated and requirees multiple Azure queries.
|
||||
/// Uses real Azure and requires [`ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME`] and related Azure cred env vars specified. Test will skip real code and pass if env vars not set.
|
||||
/// See `Azure_pagination_should_work` for more information.
|
||||
///
|
||||
/// First, create a set of Azure objects with keys `random_prefix/folder{j}/blob_{i}.txt` in [`upload_azure_data`]
|
||||
/// Then performs the following queries:
|
||||
/// 1. `list_files(None)`. This should return all files `random_prefix/folder{j}/blob_{i}.txt`
|
||||
/// 2. `list_files("folder1")`. This should return all files `random_prefix/folder1/blob_{i}.txt`
|
||||
#[test_context(MaybeEnabledAzureWithSimpleTestBlobs)]
|
||||
#[tokio::test]
|
||||
async fn azure_list_files_works(
|
||||
ctx: &mut MaybeEnabledAzureWithSimpleTestBlobs,
|
||||
) -> anyhow::Result<()> {
|
||||
let ctx = match ctx {
|
||||
MaybeEnabledAzureWithSimpleTestBlobs::Enabled(ctx) => ctx,
|
||||
MaybeEnabledAzureWithSimpleTestBlobs::Disabled => return Ok(()),
|
||||
MaybeEnabledAzureWithSimpleTestBlobs::UploadsFailed(e, _) => {
|
||||
anyhow::bail!("Azure init failed: {e:?}")
|
||||
}
|
||||
};
|
||||
let test_client = Arc::clone(&ctx.enabled.client);
|
||||
let base_prefix =
|
||||
RemotePath::new(Utf8Path::new("folder1")).context("common_prefix construction")?;
|
||||
let root_files = test_client
|
||||
.list_files(None)
|
||||
.await
|
||||
.context("client list root files failure")?
|
||||
.into_iter()
|
||||
.collect::<HashSet<_>>();
|
||||
assert_eq!(
|
||||
root_files,
|
||||
ctx.remote_blobs.clone(),
|
||||
"remote storage list_files on root mismatches with the uploads."
|
||||
);
|
||||
let nested_remote_files = test_client
|
||||
.list_files(Some(&base_prefix))
|
||||
.await
|
||||
.context("client list nested files failure")?
|
||||
.into_iter()
|
||||
.collect::<HashSet<_>>();
|
||||
let trim_remote_blobs: HashSet<_> = ctx
|
||||
.remote_blobs
|
||||
.iter()
|
||||
.map(|x| x.get_path())
|
||||
.filter(|x| x.starts_with("folder1"))
|
||||
.map(|x| RemotePath::new(x).expect("must be valid path"))
|
||||
.collect();
|
||||
assert_eq!(
|
||||
nested_remote_files, trim_remote_blobs,
|
||||
"remote storage list_files on subdirrectory mismatches with the uploads."
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test_context(MaybeEnabledAzure)]
|
||||
#[tokio::test]
|
||||
async fn azure_delete_non_exising_works(ctx: &mut MaybeEnabledAzure) -> anyhow::Result<()> {
|
||||
let ctx = match ctx {
|
||||
MaybeEnabledAzure::Enabled(ctx) => ctx,
|
||||
MaybeEnabledAzure::Disabled => return Ok(()),
|
||||
};
|
||||
|
||||
let path = RemotePath::new(Utf8Path::new(
|
||||
format!("{}/for_sure_there_is_nothing_there_really", ctx.base_prefix).as_str(),
|
||||
))
|
||||
.with_context(|| "RemotePath conversion")?;
|
||||
|
||||
ctx.client.delete(&path).await.expect("should succeed");
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test_context(MaybeEnabledAzure)]
|
||||
#[tokio::test]
|
||||
async fn azure_delete_objects_works(ctx: &mut MaybeEnabledAzure) -> anyhow::Result<()> {
|
||||
let ctx = match ctx {
|
||||
MaybeEnabledAzure::Enabled(ctx) => ctx,
|
||||
MaybeEnabledAzure::Disabled => return Ok(()),
|
||||
};
|
||||
|
||||
let path1 = RemotePath::new(Utf8Path::new(format!("{}/path1", ctx.base_prefix).as_str()))
|
||||
.with_context(|| "RemotePath conversion")?;
|
||||
|
||||
let path2 = RemotePath::new(Utf8Path::new(format!("{}/path2", ctx.base_prefix).as_str()))
|
||||
.with_context(|| "RemotePath conversion")?;
|
||||
|
||||
let path3 = RemotePath::new(Utf8Path::new(format!("{}/path3", ctx.base_prefix).as_str()))
|
||||
.with_context(|| "RemotePath conversion")?;
|
||||
|
||||
let data1 = "remote blob data1".as_bytes();
|
||||
let data1_len = data1.len();
|
||||
let data2 = "remote blob data2".as_bytes();
|
||||
let data2_len = data2.len();
|
||||
let data3 = "remote blob data3".as_bytes();
|
||||
let data3_len = data3.len();
|
||||
ctx.client
|
||||
.upload(std::io::Cursor::new(data1), data1_len, &path1, None)
|
||||
.await?;
|
||||
|
||||
ctx.client
|
||||
.upload(std::io::Cursor::new(data2), data2_len, &path2, None)
|
||||
.await?;
|
||||
|
||||
ctx.client
|
||||
.upload(std::io::Cursor::new(data3), data3_len, &path3, None)
|
||||
.await?;
|
||||
|
||||
ctx.client.delete_objects(&[path1, path2]).await?;
|
||||
|
||||
let prefixes = ctx.client.list_prefixes(None).await?;
|
||||
|
||||
assert_eq!(prefixes.len(), 1);
|
||||
|
||||
ctx.client.delete_objects(&[path3]).await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test_context(MaybeEnabledAzure)]
|
||||
#[tokio::test]
|
||||
async fn azure_upload_download_works(ctx: &mut MaybeEnabledAzure) -> anyhow::Result<()> {
|
||||
let MaybeEnabledAzure::Enabled(ctx) = ctx else {
|
||||
return Ok(());
|
||||
};
|
||||
|
||||
let path = RemotePath::new(Utf8Path::new(format!("{}/file", ctx.base_prefix).as_str()))
|
||||
.with_context(|| "RemotePath conversion")?;
|
||||
|
||||
let data = "remote blob data here".as_bytes();
|
||||
let data_len = data.len() as u64;
|
||||
|
||||
ctx.client
|
||||
.upload(std::io::Cursor::new(data), data.len(), &path, None)
|
||||
.await?;
|
||||
|
||||
async fn download_and_compare(mut dl: Download) -> anyhow::Result<Vec<u8>> {
|
||||
let mut buf = Vec::new();
|
||||
tokio::io::copy(&mut dl.download_stream, &mut buf).await?;
|
||||
Ok(buf)
|
||||
}
|
||||
// Normal download request
|
||||
let dl = ctx.client.download(&path).await?;
|
||||
let buf = download_and_compare(dl).await?;
|
||||
assert_eq!(buf, data);
|
||||
|
||||
// Full range (end specified)
|
||||
let dl = ctx
|
||||
.client
|
||||
.download_byte_range(&path, 0, Some(data_len))
|
||||
.await?;
|
||||
let buf = download_and_compare(dl).await?;
|
||||
assert_eq!(buf, data);
|
||||
|
||||
// partial range (end specified)
|
||||
let dl = ctx.client.download_byte_range(&path, 4, Some(10)).await?;
|
||||
let buf = download_and_compare(dl).await?;
|
||||
assert_eq!(buf, data[4..10]);
|
||||
|
||||
// partial range (end beyond real end)
|
||||
let dl = ctx
|
||||
.client
|
||||
.download_byte_range(&path, 8, Some(data_len * 100))
|
||||
.await?;
|
||||
let buf = download_and_compare(dl).await?;
|
||||
assert_eq!(buf, data[8..]);
|
||||
|
||||
// Partial range (end unspecified)
|
||||
let dl = ctx.client.download_byte_range(&path, 4, None).await?;
|
||||
let buf = download_and_compare(dl).await?;
|
||||
assert_eq!(buf, data[4..]);
|
||||
|
||||
// Full range (end unspecified)
|
||||
let dl = ctx.client.download_byte_range(&path, 0, None).await?;
|
||||
let buf = download_and_compare(dl).await?;
|
||||
assert_eq!(buf, data);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn ensure_logging_ready() {
|
||||
LOGGING_DONE.get_or_init(|| {
|
||||
utils::logging::init(
|
||||
utils::logging::LogFormat::Test,
|
||||
utils::logging::TracingErrorLayerEnablement::Disabled,
|
||||
)
|
||||
.expect("logging init failed");
|
||||
});
|
||||
}
|
||||
|
||||
struct EnabledAzure {
|
||||
client: Arc<GenericRemoteStorage>,
|
||||
base_prefix: &'static str,
|
||||
}
|
||||
|
||||
impl EnabledAzure {
|
||||
async fn setup(max_keys_in_list_response: Option<i32>) -> Self {
|
||||
let client = create_azure_client(max_keys_in_list_response)
|
||||
.context("Azure client creation")
|
||||
.expect("Azure client creation failed");
|
||||
|
||||
EnabledAzure {
|
||||
client,
|
||||
base_prefix: BASE_PREFIX,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
enum MaybeEnabledAzure {
|
||||
Enabled(EnabledAzure),
|
||||
Disabled,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl AsyncTestContext for MaybeEnabledAzure {
|
||||
async fn setup() -> Self {
|
||||
ensure_logging_ready();
|
||||
|
||||
if env::var(ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME).is_err() {
|
||||
info!(
|
||||
"`{}` env variable is not set, skipping the test",
|
||||
ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME
|
||||
);
|
||||
return Self::Disabled;
|
||||
}
|
||||
|
||||
Self::Enabled(EnabledAzure::setup(None).await)
|
||||
}
|
||||
}
|
||||
|
||||
enum MaybeEnabledAzureWithTestBlobs {
|
||||
Enabled(AzureWithTestBlobs),
|
||||
Disabled,
|
||||
UploadsFailed(anyhow::Error, AzureWithTestBlobs),
|
||||
}
|
||||
|
||||
struct AzureWithTestBlobs {
|
||||
enabled: EnabledAzure,
|
||||
remote_prefixes: HashSet<RemotePath>,
|
||||
remote_blobs: HashSet<RemotePath>,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl AsyncTestContext for MaybeEnabledAzureWithTestBlobs {
|
||||
async fn setup() -> Self {
|
||||
ensure_logging_ready();
|
||||
if env::var(ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME).is_err() {
|
||||
info!(
|
||||
"`{}` env variable is not set, skipping the test",
|
||||
ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME
|
||||
);
|
||||
return Self::Disabled;
|
||||
}
|
||||
|
||||
let max_keys_in_list_response = 10;
|
||||
let upload_tasks_count = 1 + (2 * usize::try_from(max_keys_in_list_response).unwrap());
|
||||
|
||||
let enabled = EnabledAzure::setup(Some(max_keys_in_list_response)).await;
|
||||
|
||||
match upload_azure_data(&enabled.client, enabled.base_prefix, upload_tasks_count).await {
|
||||
ControlFlow::Continue(uploads) => {
|
||||
info!("Remote objects created successfully");
|
||||
|
||||
Self::Enabled(AzureWithTestBlobs {
|
||||
enabled,
|
||||
remote_prefixes: uploads.prefixes,
|
||||
remote_blobs: uploads.blobs,
|
||||
})
|
||||
}
|
||||
ControlFlow::Break(uploads) => Self::UploadsFailed(
|
||||
anyhow::anyhow!("One or multiple blobs failed to upload to Azure"),
|
||||
AzureWithTestBlobs {
|
||||
enabled,
|
||||
remote_prefixes: uploads.prefixes,
|
||||
remote_blobs: uploads.blobs,
|
||||
},
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
async fn teardown(self) {
|
||||
match self {
|
||||
Self::Disabled => {}
|
||||
Self::Enabled(ctx) | Self::UploadsFailed(_, ctx) => {
|
||||
cleanup(&ctx.enabled.client, ctx.remote_blobs).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOTE: the setups for the list_prefixes test and the list_files test are very similar
|
||||
// However, they are not idential. The list_prefixes function is concerned with listing prefixes,
|
||||
// whereas the list_files function is concerned with listing files.
|
||||
// See `RemoteStorage::list_files` documentation for more details
|
||||
enum MaybeEnabledAzureWithSimpleTestBlobs {
|
||||
Enabled(AzureWithSimpleTestBlobs),
|
||||
Disabled,
|
||||
UploadsFailed(anyhow::Error, AzureWithSimpleTestBlobs),
|
||||
}
|
||||
struct AzureWithSimpleTestBlobs {
|
||||
enabled: EnabledAzure,
|
||||
remote_blobs: HashSet<RemotePath>,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl AsyncTestContext for MaybeEnabledAzureWithSimpleTestBlobs {
|
||||
async fn setup() -> Self {
|
||||
ensure_logging_ready();
|
||||
if env::var(ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME).is_err() {
|
||||
info!(
|
||||
"`{}` env variable is not set, skipping the test",
|
||||
ENABLE_REAL_AZURE_REMOTE_STORAGE_ENV_VAR_NAME
|
||||
);
|
||||
return Self::Disabled;
|
||||
}
|
||||
|
||||
let max_keys_in_list_response = 10;
|
||||
let upload_tasks_count = 1 + (2 * usize::try_from(max_keys_in_list_response).unwrap());
|
||||
|
||||
let enabled = EnabledAzure::setup(Some(max_keys_in_list_response)).await;
|
||||
|
||||
match upload_simple_azure_data(&enabled.client, upload_tasks_count).await {
|
||||
ControlFlow::Continue(uploads) => {
|
||||
info!("Remote objects created successfully");
|
||||
|
||||
Self::Enabled(AzureWithSimpleTestBlobs {
|
||||
enabled,
|
||||
remote_blobs: uploads,
|
||||
})
|
||||
}
|
||||
ControlFlow::Break(uploads) => Self::UploadsFailed(
|
||||
anyhow::anyhow!("One or multiple blobs failed to upload to Azure"),
|
||||
AzureWithSimpleTestBlobs {
|
||||
enabled,
|
||||
remote_blobs: uploads,
|
||||
},
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
async fn teardown(self) {
|
||||
match self {
|
||||
Self::Disabled => {}
|
||||
Self::Enabled(ctx) | Self::UploadsFailed(_, ctx) => {
|
||||
cleanup(&ctx.enabled.client, ctx.remote_blobs).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn create_azure_client(
|
||||
max_keys_per_list_response: Option<i32>,
|
||||
) -> anyhow::Result<Arc<GenericRemoteStorage>> {
|
||||
use rand::Rng;
|
||||
|
||||
let remote_storage_azure_container = env::var("REMOTE_STORAGE_AZURE_CONTAINER").context(
|
||||
"`REMOTE_STORAGE_AZURE_CONTAINER` env var is not set, but real Azure tests are enabled",
|
||||
)?;
|
||||
let remote_storage_azure_region = env::var("REMOTE_STORAGE_AZURE_REGION").context(
|
||||
"`REMOTE_STORAGE_AZURE_REGION` env var is not set, but real Azure tests are enabled",
|
||||
)?;
|
||||
|
||||
// due to how time works, we've had test runners use the same nanos as bucket prefixes.
|
||||
// millis is just a debugging aid for easier finding the prefix later.
|
||||
let millis = std::time::SystemTime::now()
|
||||
.duration_since(UNIX_EPOCH)
|
||||
.context("random Azure test prefix part calculation")?
|
||||
.as_millis();
|
||||
|
||||
// because nanos can be the same for two threads so can millis, add randomness
|
||||
let random = rand::thread_rng().gen::<u32>();
|
||||
|
||||
let remote_storage_config = RemoteStorageConfig {
|
||||
max_concurrent_syncs: NonZeroUsize::new(100).unwrap(),
|
||||
max_sync_errors: NonZeroU32::new(5).unwrap(),
|
||||
storage: RemoteStorageKind::AzureContainer(AzureConfig {
|
||||
container_name: remote_storage_azure_container,
|
||||
container_region: remote_storage_azure_region,
|
||||
prefix_in_container: Some(format!("test_{millis}_{random:08x}/")),
|
||||
concurrency_limit: NonZeroUsize::new(100).unwrap(),
|
||||
max_keys_per_list_response,
|
||||
}),
|
||||
};
|
||||
Ok(Arc::new(
|
||||
GenericRemoteStorage::from_config(&remote_storage_config).context("remote storage init")?,
|
||||
))
|
||||
}
|
||||
|
||||
struct Uploads {
|
||||
prefixes: HashSet<RemotePath>,
|
||||
blobs: HashSet<RemotePath>,
|
||||
}
|
||||
|
||||
async fn upload_azure_data(
|
||||
client: &Arc<GenericRemoteStorage>,
|
||||
base_prefix_str: &'static str,
|
||||
upload_tasks_count: usize,
|
||||
) -> ControlFlow<Uploads, Uploads> {
|
||||
info!("Creating {upload_tasks_count} Azure files");
|
||||
let mut upload_tasks = JoinSet::new();
|
||||
for i in 1..upload_tasks_count + 1 {
|
||||
let task_client = Arc::clone(client);
|
||||
upload_tasks.spawn(async move {
|
||||
let prefix = format!("{base_prefix_str}/sub_prefix_{i}/");
|
||||
let blob_prefix = RemotePath::new(Utf8Path::new(&prefix))
|
||||
.with_context(|| format!("{prefix:?} to RemotePath conversion"))?;
|
||||
let blob_path = blob_prefix.join(Utf8Path::new(&format!("blob_{i}")));
|
||||
debug!("Creating remote item {i} at path {blob_path:?}");
|
||||
|
||||
let data = format!("remote blob data {i}").into_bytes();
|
||||
let data_len = data.len();
|
||||
task_client
|
||||
.upload(std::io::Cursor::new(data), data_len, &blob_path, None)
|
||||
.await?;
|
||||
|
||||
Ok::<_, anyhow::Error>((blob_prefix, blob_path))
|
||||
});
|
||||
}
|
||||
|
||||
let mut upload_tasks_failed = false;
|
||||
let mut uploaded_prefixes = HashSet::with_capacity(upload_tasks_count);
|
||||
let mut uploaded_blobs = HashSet::with_capacity(upload_tasks_count);
|
||||
while let Some(task_run_result) = upload_tasks.join_next().await {
|
||||
match task_run_result
|
||||
.context("task join failed")
|
||||
.and_then(|task_result| task_result.context("upload task failed"))
|
||||
{
|
||||
Ok((upload_prefix, upload_path)) => {
|
||||
uploaded_prefixes.insert(upload_prefix);
|
||||
uploaded_blobs.insert(upload_path);
|
||||
}
|
||||
Err(e) => {
|
||||
error!("Upload task failed: {e:?}");
|
||||
upload_tasks_failed = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let uploads = Uploads {
|
||||
prefixes: uploaded_prefixes,
|
||||
blobs: uploaded_blobs,
|
||||
};
|
||||
if upload_tasks_failed {
|
||||
ControlFlow::Break(uploads)
|
||||
} else {
|
||||
ControlFlow::Continue(uploads)
|
||||
}
|
||||
}
|
||||
|
||||
async fn cleanup(client: &Arc<GenericRemoteStorage>, objects_to_delete: HashSet<RemotePath>) {
|
||||
info!(
|
||||
"Removing {} objects from the remote storage during cleanup",
|
||||
objects_to_delete.len()
|
||||
);
|
||||
let mut delete_tasks = JoinSet::new();
|
||||
for object_to_delete in objects_to_delete {
|
||||
let task_client = Arc::clone(client);
|
||||
delete_tasks.spawn(async move {
|
||||
debug!("Deleting remote item at path {object_to_delete:?}");
|
||||
task_client
|
||||
.delete(&object_to_delete)
|
||||
.await
|
||||
.with_context(|| format!("{object_to_delete:?} removal"))
|
||||
});
|
||||
}
|
||||
|
||||
while let Some(task_run_result) = delete_tasks.join_next().await {
|
||||
match task_run_result {
|
||||
Ok(task_result) => match task_result {
|
||||
Ok(()) => {}
|
||||
Err(e) => error!("Delete task failed: {e:?}"),
|
||||
},
|
||||
Err(join_err) => error!("Delete task did not finish correctly: {join_err}"),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Uploads files `folder{j}/blob{i}.txt`. See test description for more details.
|
||||
async fn upload_simple_azure_data(
|
||||
client: &Arc<GenericRemoteStorage>,
|
||||
upload_tasks_count: usize,
|
||||
) -> ControlFlow<HashSet<RemotePath>, HashSet<RemotePath>> {
|
||||
info!("Creating {upload_tasks_count} Azure files");
|
||||
let mut upload_tasks = JoinSet::new();
|
||||
for i in 1..upload_tasks_count + 1 {
|
||||
let task_client = Arc::clone(client);
|
||||
upload_tasks.spawn(async move {
|
||||
let blob_path = PathBuf::from(format!("folder{}/blob_{}.txt", i / 7, i));
|
||||
let blob_path = RemotePath::new(
|
||||
Utf8Path::from_path(blob_path.as_path()).expect("must be valid blob path"),
|
||||
)
|
||||
.with_context(|| format!("{blob_path:?} to RemotePath conversion"))?;
|
||||
debug!("Creating remote item {i} at path {blob_path:?}");
|
||||
|
||||
let data = format!("remote blob data {i}").into_bytes();
|
||||
let data_len = data.len();
|
||||
task_client
|
||||
.upload(std::io::Cursor::new(data), data_len, &blob_path, None)
|
||||
.await?;
|
||||
|
||||
Ok::<_, anyhow::Error>(blob_path)
|
||||
});
|
||||
}
|
||||
|
||||
let mut upload_tasks_failed = false;
|
||||
let mut uploaded_blobs = HashSet::with_capacity(upload_tasks_count);
|
||||
while let Some(task_run_result) = upload_tasks.join_next().await {
|
||||
match task_run_result
|
||||
.context("task join failed")
|
||||
.and_then(|task_result| task_result.context("upload task failed"))
|
||||
{
|
||||
Ok(upload_path) => {
|
||||
uploaded_blobs.insert(upload_path);
|
||||
}
|
||||
Err(e) => {
|
||||
error!("Upload task failed: {e:?}");
|
||||
upload_tasks_failed = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if upload_tasks_failed {
|
||||
ControlFlow::Break(uploaded_blobs)
|
||||
} else {
|
||||
ControlFlow::Continue(uploaded_blobs)
|
||||
}
|
||||
}
|
||||
@@ -31,13 +31,14 @@ futures = { version = "0.3" }
|
||||
futures-channel = { version = "0.3", features = ["sink"] }
|
||||
futures-core = { version = "0.3" }
|
||||
futures-executor = { version = "0.3" }
|
||||
futures-io = { version = "0.3" }
|
||||
futures-sink = { version = "0.3" }
|
||||
futures-util = { version = "0.3", features = ["channel", "io", "sink"] }
|
||||
hex = { version = "0.4", features = ["serde"] }
|
||||
hyper = { version = "0.14", features = ["full"] }
|
||||
itertools = { version = "0.10" }
|
||||
libc = { version = "0.2", features = ["extra_traits"] }
|
||||
log = { version = "0.4", default-features = false, features = ["std"] }
|
||||
log = { version = "0.4", default-features = false, features = ["kv_unstable", "std"] }
|
||||
memchr = { version = "2" }
|
||||
nom = { version = "7" }
|
||||
num-bigint = { version = "0.4" }
|
||||
@@ -47,7 +48,7 @@ prost = { version = "0.11" }
|
||||
rand = { version = "0.8", features = ["small_rng"] }
|
||||
regex = { version = "1" }
|
||||
regex-syntax = { version = "0.7" }
|
||||
reqwest = { version = "0.11", default-features = false, features = ["blocking", "json", "multipart", "rustls-tls"] }
|
||||
reqwest = { version = "0.11", default-features = false, features = ["blocking", "default-tls", "json", "multipart", "rustls-tls", "stream"] }
|
||||
ring = { version = "0.16", features = ["std"] }
|
||||
rustls = { version = "0.21", features = ["dangerous_configuration"] }
|
||||
scopeguard = { version = "1" }
|
||||
@@ -55,7 +56,8 @@ serde = { version = "1", features = ["alloc", "derive"] }
|
||||
serde_json = { version = "1", features = ["raw_value"] }
|
||||
smallvec = { version = "1", default-features = false, features = ["write"] }
|
||||
socket2 = { version = "0.4", default-features = false, features = ["all"] }
|
||||
time = { version = "0.3", features = ["formatting", "macros", "parsing"] }
|
||||
standback = { version = "0.2", default-features = false, features = ["std"] }
|
||||
time = { version = "0.3", features = ["macros", "serde-well-known"] }
|
||||
tokio = { version = "1", features = ["fs", "io-std", "io-util", "macros", "net", "process", "rt-multi-thread", "signal", "test-util"] }
|
||||
tokio-rustls = { version = "0.24" }
|
||||
tokio-util = { version = "0.7", features = ["codec", "io"] }
|
||||
@@ -75,14 +77,16 @@ cc = { version = "1", default-features = false, features = ["parallel"] }
|
||||
either = { version = "1" }
|
||||
itertools = { version = "0.10" }
|
||||
libc = { version = "0.2", features = ["extra_traits"] }
|
||||
log = { version = "0.4", default-features = false, features = ["std"] }
|
||||
log = { version = "0.4", default-features = false, features = ["kv_unstable", "std"] }
|
||||
memchr = { version = "2" }
|
||||
nom = { version = "7" }
|
||||
prost = { version = "0.11" }
|
||||
regex = { version = "1" }
|
||||
regex-syntax = { version = "0.7" }
|
||||
serde = { version = "1", features = ["alloc", "derive"] }
|
||||
standback = { version = "0.2", default-features = false, features = ["std"] }
|
||||
syn-dff4ba8e3ae991db = { package = "syn", version = "1", features = ["extra-traits", "full", "visit"] }
|
||||
syn-f595c2ba2a3f28df = { package = "syn", version = "2", features = ["extra-traits", "full", "visit", "visit-mut"] }
|
||||
time-macros = { version = "0.2", default-features = false, features = ["formatting", "parsing", "serde"] }
|
||||
|
||||
### END HAKARI SECTION
|
||||
|
||||
Reference in New Issue
Block a user