mirror of
https://github.com/neondatabase/neon.git
synced 2025-12-23 06:09:59 +00:00
Switch tenant snapshot subcommand to remote_storage (#11685)
Switches the tenant snapshot subcommand of the storage scrubber to `remote_storage`. As this is the last piece of the storage scrubber still using the S3 SDK, this finishes the project started in #7547. This allows us to do tenant snapshots on Azure as well. Builds on #11671 Fixes #8830
This commit is contained in:
2
Cargo.lock
generated
2
Cargo.lock
generated
@@ -6711,8 +6711,6 @@ version = "0.1.0"
|
|||||||
dependencies = [
|
dependencies = [
|
||||||
"anyhow",
|
"anyhow",
|
||||||
"async-stream",
|
"async-stream",
|
||||||
"aws-config",
|
|
||||||
"aws-sdk-s3",
|
|
||||||
"camino",
|
"camino",
|
||||||
"chrono",
|
"chrono",
|
||||||
"clap",
|
"clap",
|
||||||
|
|||||||
@@ -5,8 +5,6 @@ edition = "2024"
|
|||||||
license.workspace = true
|
license.workspace = true
|
||||||
|
|
||||||
[dependencies]
|
[dependencies]
|
||||||
aws-config.workspace = true
|
|
||||||
aws-sdk-s3.workspace = true
|
|
||||||
either.workspace = true
|
either.workspace = true
|
||||||
anyhow.workspace = true
|
anyhow.workspace = true
|
||||||
hex.workspace = true
|
hex.workspace = true
|
||||||
|
|||||||
@@ -12,14 +12,9 @@ pub mod tenant_snapshot;
|
|||||||
|
|
||||||
use std::env;
|
use std::env;
|
||||||
use std::fmt::Display;
|
use std::fmt::Display;
|
||||||
use std::sync::Arc;
|
|
||||||
use std::time::{Duration, SystemTime};
|
use std::time::{Duration, SystemTime};
|
||||||
|
|
||||||
use anyhow::Context;
|
use anyhow::Context;
|
||||||
use aws_config::retry::{RetryConfigBuilder, RetryMode};
|
|
||||||
use aws_sdk_s3::Client;
|
|
||||||
use aws_sdk_s3::config::Region;
|
|
||||||
use aws_sdk_s3::error::DisplayErrorContext;
|
|
||||||
use camino::{Utf8Path, Utf8PathBuf};
|
use camino::{Utf8Path, Utf8PathBuf};
|
||||||
use clap::ValueEnum;
|
use clap::ValueEnum;
|
||||||
use futures::{Stream, StreamExt};
|
use futures::{Stream, StreamExt};
|
||||||
@@ -28,7 +23,7 @@ use pageserver::tenant::remote_timeline_client::{remote_tenant_path, remote_time
|
|||||||
use pageserver_api::shard::TenantShardId;
|
use pageserver_api::shard::TenantShardId;
|
||||||
use remote_storage::{
|
use remote_storage::{
|
||||||
DownloadOpts, GenericRemoteStorage, Listing, ListingMode, RemotePath, RemoteStorageConfig,
|
DownloadOpts, GenericRemoteStorage, Listing, ListingMode, RemotePath, RemoteStorageConfig,
|
||||||
RemoteStorageKind, S3Config,
|
RemoteStorageKind, VersionId,
|
||||||
};
|
};
|
||||||
use reqwest::Url;
|
use reqwest::Url;
|
||||||
use serde::{Deserialize, Serialize};
|
use serde::{Deserialize, Serialize};
|
||||||
@@ -351,21 +346,6 @@ pub fn init_logging(file_name: &str) -> Option<WorkerGuard> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn init_s3_client(bucket_region: Region) -> Client {
|
|
||||||
let mut retry_config_builder = RetryConfigBuilder::new();
|
|
||||||
|
|
||||||
retry_config_builder
|
|
||||||
.set_max_attempts(Some(3))
|
|
||||||
.set_mode(Some(RetryMode::Adaptive));
|
|
||||||
|
|
||||||
let config = aws_config::defaults(aws_config::BehaviorVersion::v2024_03_28())
|
|
||||||
.region(bucket_region)
|
|
||||||
.retry_config(retry_config_builder.build())
|
|
||||||
.load()
|
|
||||||
.await;
|
|
||||||
Client::new(&config)
|
|
||||||
}
|
|
||||||
|
|
||||||
fn default_prefix_in_bucket(node_kind: NodeKind) -> &'static str {
|
fn default_prefix_in_bucket(node_kind: NodeKind) -> &'static str {
|
||||||
match node_kind {
|
match node_kind {
|
||||||
NodeKind::Pageserver => "pageserver/v1/",
|
NodeKind::Pageserver => "pageserver/v1/",
|
||||||
@@ -385,23 +365,6 @@ fn make_root_target(desc_str: String, prefix_in_bucket: String, node_kind: NodeK
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn init_remote_s3(
|
|
||||||
bucket_config: S3Config,
|
|
||||||
node_kind: NodeKind,
|
|
||||||
) -> anyhow::Result<(Arc<Client>, RootTarget)> {
|
|
||||||
let bucket_region = Region::new(bucket_config.bucket_region);
|
|
||||||
let s3_client = Arc::new(init_s3_client(bucket_region).await);
|
|
||||||
let default_prefix = default_prefix_in_bucket(node_kind).to_string();
|
|
||||||
|
|
||||||
let s3_root = make_root_target(
|
|
||||||
bucket_config.bucket_name,
|
|
||||||
bucket_config.prefix_in_bucket.unwrap_or(default_prefix),
|
|
||||||
node_kind,
|
|
||||||
);
|
|
||||||
|
|
||||||
Ok((s3_client, s3_root))
|
|
||||||
}
|
|
||||||
|
|
||||||
async fn init_remote(
|
async fn init_remote(
|
||||||
mut storage_config: BucketConfig,
|
mut storage_config: BucketConfig,
|
||||||
node_kind: NodeKind,
|
node_kind: NodeKind,
|
||||||
@@ -499,7 +462,7 @@ async fn list_objects_with_retries(
|
|||||||
remote_client.bucket_name().unwrap_or_default(),
|
remote_client.bucket_name().unwrap_or_default(),
|
||||||
s3_target.prefix_in_bucket,
|
s3_target.prefix_in_bucket,
|
||||||
s3_target.delimiter,
|
s3_target.delimiter,
|
||||||
DisplayErrorContext(e),
|
e,
|
||||||
);
|
);
|
||||||
let backoff_time = 1 << trial.min(5);
|
let backoff_time = 1 << trial.min(5);
|
||||||
tokio::time::sleep(Duration::from_secs(backoff_time)).await;
|
tokio::time::sleep(Duration::from_secs(backoff_time)).await;
|
||||||
@@ -549,14 +512,18 @@ async fn download_object_with_retries(
|
|||||||
anyhow::bail!("Failed to download objects with key {key} {MAX_RETRIES} times")
|
anyhow::bail!("Failed to download objects with key {key} {MAX_RETRIES} times")
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn download_object_to_file_s3(
|
async fn download_object_to_file(
|
||||||
s3_client: &Client,
|
remote_storage: &GenericRemoteStorage,
|
||||||
bucket_name: &str,
|
key: &RemotePath,
|
||||||
key: &str,
|
version_id: Option<VersionId>,
|
||||||
version_id: Option<&str>,
|
|
||||||
local_path: &Utf8Path,
|
local_path: &Utf8Path,
|
||||||
) -> anyhow::Result<()> {
|
) -> anyhow::Result<()> {
|
||||||
|
let opts = DownloadOpts {
|
||||||
|
version_id: version_id.clone(),
|
||||||
|
..Default::default()
|
||||||
|
};
|
||||||
let tmp_path = Utf8PathBuf::from(format!("{local_path}.tmp"));
|
let tmp_path = Utf8PathBuf::from(format!("{local_path}.tmp"));
|
||||||
|
let cancel = CancellationToken::new();
|
||||||
for _ in 0..MAX_RETRIES {
|
for _ in 0..MAX_RETRIES {
|
||||||
tokio::fs::remove_file(&tmp_path)
|
tokio::fs::remove_file(&tmp_path)
|
||||||
.await
|
.await
|
||||||
@@ -566,28 +533,24 @@ async fn download_object_to_file_s3(
|
|||||||
.await
|
.await
|
||||||
.context("Opening output file")?;
|
.context("Opening output file")?;
|
||||||
|
|
||||||
let request = s3_client.get_object().bucket(bucket_name).key(key);
|
let res = remote_storage.download(key, &opts, &cancel).await;
|
||||||
|
|
||||||
let request = match version_id {
|
let download = match res {
|
||||||
Some(version_id) => request.version_id(version_id),
|
|
||||||
None => request,
|
|
||||||
};
|
|
||||||
|
|
||||||
let response_stream = match request.send().await {
|
|
||||||
Ok(response) => response,
|
Ok(response) => response,
|
||||||
Err(e) => {
|
Err(e) => {
|
||||||
error!(
|
error!(
|
||||||
"Failed to download object for key {key} version {}: {e:#}",
|
"Failed to download object for key {key} version {:?}: {e:#}",
|
||||||
version_id.unwrap_or("")
|
&version_id.as_ref().unwrap_or(&VersionId(String::new()))
|
||||||
);
|
);
|
||||||
tokio::time::sleep(Duration::from_secs(1)).await;
|
tokio::time::sleep(Duration::from_secs(1)).await;
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
let mut read_stream = response_stream.body.into_async_read();
|
//response_stream.download_stream
|
||||||
|
|
||||||
tokio::io::copy(&mut read_stream, &mut file).await?;
|
let mut body = tokio_util::io::StreamReader::new(download.download_stream);
|
||||||
|
tokio::io::copy(&mut body, &mut file).await?;
|
||||||
|
|
||||||
tokio::fs::rename(&tmp_path, local_path).await?;
|
tokio::fs::rename(&tmp_path, local_path).await?;
|
||||||
return Ok(());
|
return Ok(());
|
||||||
|
|||||||
@@ -1,31 +1,30 @@
|
|||||||
use std::collections::HashMap;
|
use std::collections::HashMap;
|
||||||
use std::sync::Arc;
|
|
||||||
|
|
||||||
use anyhow::Context;
|
use anyhow::Context;
|
||||||
use async_stream::stream;
|
use async_stream::stream;
|
||||||
use aws_sdk_s3::Client;
|
|
||||||
use camino::Utf8PathBuf;
|
use camino::Utf8PathBuf;
|
||||||
use futures::{StreamExt, TryStreamExt};
|
use futures::{StreamExt, TryStreamExt};
|
||||||
use pageserver::tenant::IndexPart;
|
use pageserver::tenant::IndexPart;
|
||||||
use pageserver::tenant::remote_timeline_client::index::LayerFileMetadata;
|
use pageserver::tenant::remote_timeline_client::index::LayerFileMetadata;
|
||||||
|
use pageserver::tenant::remote_timeline_client::remote_layer_path;
|
||||||
use pageserver::tenant::storage_layer::LayerName;
|
use pageserver::tenant::storage_layer::LayerName;
|
||||||
use pageserver_api::shard::TenantShardId;
|
use pageserver_api::shard::TenantShardId;
|
||||||
use remote_storage::{GenericRemoteStorage, S3Config};
|
use remote_storage::GenericRemoteStorage;
|
||||||
|
use tokio_util::sync::CancellationToken;
|
||||||
use utils::generation::Generation;
|
use utils::generation::Generation;
|
||||||
use utils::id::TenantId;
|
use utils::id::TenantId;
|
||||||
|
|
||||||
use crate::checks::{BlobDataParseResult, RemoteTimelineBlobData, list_timeline_blobs};
|
use crate::checks::{BlobDataParseResult, RemoteTimelineBlobData, list_timeline_blobs};
|
||||||
use crate::metadata_stream::{stream_tenant_shards, stream_tenant_timelines};
|
use crate::metadata_stream::{stream_tenant_shards, stream_tenant_timelines};
|
||||||
use crate::{
|
use crate::{
|
||||||
BucketConfig, NodeKind, RootTarget, TenantShardTimelineId, download_object_to_file_s3,
|
BucketConfig, NodeKind, RootTarget, TenantShardTimelineId, download_object_to_file, init_remote,
|
||||||
init_remote, init_remote_s3,
|
|
||||||
};
|
};
|
||||||
|
|
||||||
pub struct SnapshotDownloader {
|
pub struct SnapshotDownloader {
|
||||||
s3_client: Arc<Client>,
|
remote_client: GenericRemoteStorage,
|
||||||
s3_root: RootTarget,
|
#[allow(dead_code)]
|
||||||
|
target: RootTarget,
|
||||||
bucket_config: BucketConfig,
|
bucket_config: BucketConfig,
|
||||||
bucket_config_s3: S3Config,
|
|
||||||
tenant_id: TenantId,
|
tenant_id: TenantId,
|
||||||
output_path: Utf8PathBuf,
|
output_path: Utf8PathBuf,
|
||||||
concurrency: usize,
|
concurrency: usize,
|
||||||
@@ -38,17 +37,13 @@ impl SnapshotDownloader {
|
|||||||
output_path: Utf8PathBuf,
|
output_path: Utf8PathBuf,
|
||||||
concurrency: usize,
|
concurrency: usize,
|
||||||
) -> anyhow::Result<Self> {
|
) -> anyhow::Result<Self> {
|
||||||
let bucket_config_s3 = match &bucket_config.0.storage {
|
let (remote_client, target) =
|
||||||
remote_storage::RemoteStorageKind::AwsS3(config) => config.clone(),
|
init_remote(bucket_config.clone(), NodeKind::Pageserver).await?;
|
||||||
_ => panic!("only S3 configuration is supported for snapshot downloading"),
|
|
||||||
};
|
|
||||||
let (s3_client, s3_root) =
|
|
||||||
init_remote_s3(bucket_config_s3.clone(), NodeKind::Pageserver).await?;
|
|
||||||
Ok(Self {
|
Ok(Self {
|
||||||
s3_client,
|
remote_client,
|
||||||
s3_root,
|
target,
|
||||||
bucket_config,
|
bucket_config,
|
||||||
bucket_config_s3,
|
|
||||||
tenant_id,
|
tenant_id,
|
||||||
output_path,
|
output_path,
|
||||||
concurrency,
|
concurrency,
|
||||||
@@ -61,6 +56,7 @@ impl SnapshotDownloader {
|
|||||||
layer_name: LayerName,
|
layer_name: LayerName,
|
||||||
layer_metadata: LayerFileMetadata,
|
layer_metadata: LayerFileMetadata,
|
||||||
) -> anyhow::Result<(LayerName, LayerFileMetadata)> {
|
) -> anyhow::Result<(LayerName, LayerFileMetadata)> {
|
||||||
|
let cancel = CancellationToken::new();
|
||||||
// Note this is local as in a local copy of S3 data, not local as in the pageserver's local format. They use
|
// Note this is local as in a local copy of S3 data, not local as in the pageserver's local format. They use
|
||||||
// different layer names (remote-style has the generation suffix)
|
// different layer names (remote-style has the generation suffix)
|
||||||
let local_path = self.output_path.join(format!(
|
let local_path = self.output_path.join(format!(
|
||||||
@@ -82,30 +78,27 @@ impl SnapshotDownloader {
|
|||||||
} else {
|
} else {
|
||||||
tracing::debug!("{} requires download...", local_path);
|
tracing::debug!("{} requires download...", local_path);
|
||||||
|
|
||||||
let timeline_root = self.s3_root.timeline_root(&ttid);
|
let remote_path = remote_layer_path(
|
||||||
let remote_layer_path = format!(
|
&ttid.tenant_shard_id.tenant_id,
|
||||||
"{}{}{}",
|
&ttid.timeline_id,
|
||||||
timeline_root.prefix_in_bucket,
|
layer_metadata.shard,
|
||||||
layer_name,
|
&layer_name,
|
||||||
layer_metadata.generation.get_suffix()
|
layer_metadata.generation,
|
||||||
);
|
);
|
||||||
|
let mode = remote_storage::ListingMode::NoDelimiter;
|
||||||
|
|
||||||
// List versions: the object might be deleted.
|
// List versions: the object might be deleted.
|
||||||
let versions = self
|
let versions = self
|
||||||
.s3_client
|
.remote_client
|
||||||
.list_object_versions()
|
.list_versions(Some(&remote_path), mode, None, &cancel)
|
||||||
.bucket(self.bucket_config_s3.bucket_name.clone())
|
|
||||||
.prefix(&remote_layer_path)
|
|
||||||
.send()
|
|
||||||
.await?;
|
.await?;
|
||||||
let Some(version) = versions.versions.as_ref().and_then(|v| v.first()) else {
|
let Some(version) = versions.versions.first() else {
|
||||||
return Err(anyhow::anyhow!("No versions found for {remote_layer_path}"));
|
return Err(anyhow::anyhow!("No versions found for {remote_path}"));
|
||||||
};
|
};
|
||||||
download_object_to_file_s3(
|
download_object_to_file(
|
||||||
&self.s3_client,
|
&self.remote_client,
|
||||||
&self.bucket_config_s3.bucket_name,
|
&remote_path,
|
||||||
&remote_layer_path,
|
version.version_id().cloned(),
|
||||||
version.version_id.as_deref(),
|
|
||||||
&local_path,
|
&local_path,
|
||||||
)
|
)
|
||||||
.await?;
|
.await?;
|
||||||
|
|||||||
Reference in New Issue
Block a user