pageserver: heatmap uploads (#6050)

Dependency (commits inline):
https://github.com/neondatabase/neon/pull/5842

## Problem

Secondary mode tenants need a manifest of what to download. Ultimately
this will be some kind of heat-scored set of layers, but as a robust
first step we will simply use the set of resident layers: secondary
tenant locations will aim to match the on-disk content of the attached
location.

## Summary of changes

- Add heatmap types representing the remote structure
- Add hooks to Tenant/Timeline for generating these heatmaps
- Create a new `HeatmapUploader` type that is external to `Tenant`, and
responsible for walking the list of attached tenants and scheduling
heatmap uploads.

Notes to reviewers:
- Putting the logic for uploads (and later, secondary mode downloads)
outside of `Tenant` is an opinionated choice, motivated by:
- Enable future smarter scheduling of operations, e.g. uploading the
stalest tenant first, rather than having all tenants compete for a fair
semaphore on a first-come-first-served basis. Similarly for downloads,
we may wish to schedule the tenants with the hottest un-downloaded
layers first.
- Enable accessing upload-related state without synchronization (it
belongs to HeatmapUploader, rather than being some Mutex<>'d part of
Tenant)
- Avoid further expanding the scope of Tenant/Timeline types, which are
already among the largest in the codebase
- You might reasonably wonder how much of the uploader code could be a
generic job manager thing. Probably some of it: but let's defer pulling
that out until we have at least two users (perhaps secondary downloads
will be the second one) to highlight which bits are really generic.

Compromises:
- Later, instead of using digests of heatmaps to decide whether anything
changed, I would prefer to avoid walking the layers in tenants that
don't have changes: tracking that will be a bit invasive, as it needs
input from both remote_timeline_client and Layer.
This commit is contained in:
John Spray
2023-12-14 13:09:24 +00:00
committed by GitHub
parent 6987b5c44e
commit c4e0ef507f
22 changed files with 1033 additions and 7 deletions

1
Cargo.lock generated
View File

@@ -3103,6 +3103,7 @@ dependencies = [
"humantime-serde",
"hyper",
"itertools",
"md5",
"metrics",
"nix 0.26.2",
"num-traits",

View File

@@ -407,6 +407,7 @@ impl PageServerNode {
.map(|x| x.parse::<bool>())
.transpose()
.context("Failed to parse 'gc_feedback' as bool")?,
heatmap_period: settings.remove("heatmap_period").map(|x| x.to_string()),
};
let request = models::TenantCreateRequest {
@@ -504,6 +505,7 @@ impl PageServerNode {
.map(|x| x.parse::<bool>())
.transpose()
.context("Failed to parse 'gc_feedback' as bool")?,
heatmap_period: settings.remove("heatmap_period").map(|x| x.to_string()),
}
};

View File

@@ -237,6 +237,7 @@ pub struct TenantConfig {
pub min_resident_size_override: Option<u64>,
pub evictions_low_residence_duration_metric_threshold: Option<String>,
pub gc_feedback: Option<bool>,
pub heatmap_period: Option<String>,
}
/// A flattened analog of a `pagesever::tenant::LocationMode`, which

View File

@@ -36,6 +36,7 @@ humantime.workspace = true
humantime-serde.workspace = true
hyper.workspace = true
itertools.workspace = true
md5.workspace = true
nix.workspace = true
# hack to get the number of worker threads tokio uses
num_cpus = { version = "1.15" }

View File

@@ -14,7 +14,7 @@ use pageserver::control_plane_client::ControlPlaneClient;
use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_eviction_task};
use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING};
use pageserver::task_mgr::WALRECEIVER_RUNTIME;
use pageserver::tenant::TenantSharedResources;
use pageserver::tenant::{secondary, TenantSharedResources};
use remote_storage::GenericRemoteStorage;
use tokio::time::Instant;
use tracing::*;
@@ -504,6 +504,17 @@ fn start_pageserver(
}
});
let secondary_controller = if let Some(remote_storage) = &remote_storage {
secondary::spawn_tasks(
tenant_manager.clone(),
remote_storage.clone(),
background_jobs_barrier.clone(),
shutdown_pageserver.clone(),
)
} else {
secondary::null_controller()
};
// shared state between the disk-usage backed eviction background task and the http endpoint
// that allows triggering disk-usage based eviction manually. note that the http endpoint
// is still accessible even if background task is not configured as long as remote storage has
@@ -533,6 +544,7 @@ fn start_pageserver(
broker_client.clone(),
disk_usage_eviction_state,
deletion_queue.new_client(),
secondary_controller,
)
.context("Failed to initialize router state")?,
);

View File

@@ -70,6 +70,8 @@ pub mod defaults {
pub const DEFAULT_SYNTHETIC_SIZE_CALCULATION_INTERVAL: &str = "10 min";
pub const DEFAULT_BACKGROUND_TASK_MAXIMUM_DELAY: &str = "10s";
pub const DEFAULT_HEATMAP_UPLOAD_CONCURRENCY: usize = 8;
///
/// Default built-in configuration file.
///
@@ -117,6 +119,8 @@ pub mod defaults {
#evictions_low_residence_duration_metric_threshold = '{DEFAULT_EVICTIONS_LOW_RESIDENCE_DURATION_METRIC_THRESHOLD}'
#gc_feedback = false
#heatmap_upload_concurrency = {DEFAULT_HEATMAP_UPLOAD_CONCURRENCY}
[remote_storage]
"#
@@ -215,6 +219,10 @@ pub struct PageServerConf {
/// If true, pageserver will make best-effort to operate without a control plane: only
/// for use in major incidents.
pub control_plane_emergency_mode: bool,
/// How many heatmap uploads may be done concurrency: lower values implicitly deprioritize
/// heatmap uploads vs. other remote storage operations.
pub heatmap_upload_concurrency: usize,
}
/// We do not want to store this in a PageServerConf because the latter may be logged
@@ -293,6 +301,8 @@ struct PageServerConfigBuilder {
control_plane_api: BuilderValue<Option<Url>>,
control_plane_api_token: BuilderValue<Option<SecretString>>,
control_plane_emergency_mode: BuilderValue<bool>,
heatmap_upload_concurrency: BuilderValue<usize>,
}
impl Default for PageServerConfigBuilder {
@@ -361,6 +371,8 @@ impl Default for PageServerConfigBuilder {
control_plane_api: Set(None),
control_plane_api_token: Set(None),
control_plane_emergency_mode: Set(false),
heatmap_upload_concurrency: Set(DEFAULT_HEATMAP_UPLOAD_CONCURRENCY),
}
}
}
@@ -501,6 +513,10 @@ impl PageServerConfigBuilder {
self.control_plane_emergency_mode = BuilderValue::Set(enabled)
}
pub fn heatmap_upload_concurrency(&mut self, value: usize) {
self.heatmap_upload_concurrency = BuilderValue::Set(value)
}
pub fn build(self) -> anyhow::Result<PageServerConf> {
let concurrent_tenant_size_logical_size_queries = self
.concurrent_tenant_size_logical_size_queries
@@ -595,6 +611,10 @@ impl PageServerConfigBuilder {
control_plane_emergency_mode: self
.control_plane_emergency_mode
.ok_or(anyhow!("missing control_plane_emergency_mode"))?,
heatmap_upload_concurrency: self
.heatmap_upload_concurrency
.ok_or(anyhow!("missing heatmap_upload_concurrency"))?,
})
}
}
@@ -828,7 +848,9 @@ impl PageServerConf {
},
"control_plane_emergency_mode" => {
builder.control_plane_emergency_mode(parse_toml_bool(key, item)?)
},
"heatmap_upload_concurrency" => {
builder.heatmap_upload_concurrency(parse_toml_u64(key, item)? as usize)
},
_ => bail!("unrecognized pageserver option '{key}'"),
}
@@ -896,6 +918,7 @@ impl PageServerConf {
control_plane_api: None,
control_plane_api_token: None,
control_plane_emergency_mode: false,
heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY,
}
}
}
@@ -1120,7 +1143,8 @@ background_task_maximum_delay = '334 s'
)?,
control_plane_api: None,
control_plane_api_token: None,
control_plane_emergency_mode: false
control_plane_emergency_mode: false,
heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY
},
"Correct defaults should be used when no config values are provided"
);
@@ -1177,7 +1201,8 @@ background_task_maximum_delay = '334 s'
background_task_maximum_delay: Duration::from_secs(334),
control_plane_api: None,
control_plane_api_token: None,
control_plane_emergency_mode: false
control_plane_emergency_mode: false,
heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY
},
"Should be able to parse all basic config values correctly"
);

View File

@@ -1405,6 +1405,8 @@ components:
type: integer
trace_read_requests:
type: boolean
heatmap_period:
type: integer
TenantConfigResponse:
type: object
properties:

View File

@@ -42,6 +42,7 @@ use crate::tenant::mgr::{
GetTenantError, SetNewTenantConfigError, TenantManager, TenantMapError, TenantMapInsertError,
TenantSlotError, TenantSlotUpsertError, TenantStateError,
};
use crate::tenant::secondary::SecondaryController;
use crate::tenant::size::ModelInputs;
use crate::tenant::storage_layer::LayerAccessStatsReset;
use crate::tenant::timeline::CompactFlags;
@@ -75,9 +76,11 @@ pub struct State {
broker_client: storage_broker::BrokerClientChannel,
disk_usage_eviction_state: Arc<disk_usage_eviction_task::State>,
deletion_queue_client: DeletionQueueClient,
secondary_controller: SecondaryController,
}
impl State {
#[allow(clippy::too_many_arguments)]
pub fn new(
conf: &'static PageServerConf,
tenant_manager: Arc<TenantManager>,
@@ -86,6 +89,7 @@ impl State {
broker_client: storage_broker::BrokerClientChannel,
disk_usage_eviction_state: Arc<disk_usage_eviction_task::State>,
deletion_queue_client: DeletionQueueClient,
secondary_controller: SecondaryController,
) -> anyhow::Result<Self> {
let allowlist_routes = ["/v1/status", "/v1/doc", "/swagger.yml", "/metrics"]
.iter()
@@ -100,6 +104,7 @@ impl State {
broker_client,
disk_usage_eviction_state,
deletion_queue_client,
secondary_controller,
})
}
@@ -1647,6 +1652,21 @@ async fn disk_usage_eviction_run(
json_response(StatusCode::OK, res)
}
async fn secondary_upload_handler(
request: Request<Body>,
_cancel: CancellationToken,
) -> Result<Response<Body>, ApiError> {
let state = get_state(&request);
let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?;
state
.secondary_controller
.upload_tenant(tenant_shard_id)
.await
.map_err(ApiError::InternalServerError)?;
json_response(StatusCode::OK, ())
}
async fn handler_404(_: Request<Body>) -> Result<Response<Body>, ApiError> {
json_response(
StatusCode::NOT_FOUND,
@@ -1906,6 +1926,9 @@ pub fn make_router(
"/v1/tenant/:tenant_shard_id/timeline/:timeline_id/layer/:layer_file_name",
|r| api_handler(r, evict_timeline_layer_handler),
)
.post("/v1/tenant/:tenant_shard_id/heatmap_upload", |r| {
api_handler(r, secondary_upload_handler)
})
.put("/v1/disk_usage_eviction/run", |r| {
api_handler(r, disk_usage_eviction_run)
})

View File

@@ -1270,6 +1270,28 @@ pub(crate) static WAL_INGEST: Lazy<WalIngestMetrics> = Lazy::new(|| WalIngestMet
)
.expect("failed to define a metric"),
});
pub(crate) struct SecondaryModeMetrics {
pub(crate) upload_heatmap: IntCounter,
pub(crate) upload_heatmap_errors: IntCounter,
pub(crate) upload_heatmap_duration: Histogram,
}
pub(crate) static SECONDARY_MODE: Lazy<SecondaryModeMetrics> = Lazy::new(|| SecondaryModeMetrics {
upload_heatmap: register_int_counter!(
"pageserver_secondary_upload_heatmap",
"Number of heatmaps written to remote storage by attached tenants"
)
.expect("failed to define a metric"),
upload_heatmap_errors: register_int_counter!(
"pageserver_secondary_upload_heatmap_errors",
"Failures writing heatmap to remote storage"
)
.expect("failed to define a metric"),
upload_heatmap_duration: register_histogram!(
"pageserver_secondary_upload_heatmap_duration",
"Time to build and upload a heatmap, including any waiting inside the S3 client"
)
.expect("failed to define a metric"),
});
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub enum RemoteOpKind {

View File

@@ -258,6 +258,9 @@ pub enum TaskKind {
/// See [`crate::disk_usage_eviction_task`].
DiskUsageEviction,
/// See [`crate::tenant::secondary`].
SecondaryUploads,
// Initial logical size calculation
InitialLogicalSizeCalculation,

View File

@@ -144,6 +144,7 @@ pub mod storage_layer;
pub mod config;
pub mod delete;
pub mod mgr;
pub mod secondary;
pub mod tasks;
pub mod upload_queue;
@@ -2114,6 +2115,14 @@ impl Tenant {
.attach_mode
.clone()
}
pub(crate) fn get_tenant_shard_id(&self) -> &TenantShardId {
&self.tenant_shard_id
}
pub(crate) fn get_generation(&self) -> Generation {
self.generation
}
}
/// Given a Vec of timelines and their ancestors (timeline_id, ancestor_id),
@@ -2252,6 +2261,18 @@ impl Tenant {
.or(self.conf.default_tenant_conf.min_resident_size_override)
}
pub fn get_heatmap_period(&self) -> Option<Duration> {
let tenant_conf = self.tenant_conf.read().unwrap().tenant_conf;
let heatmap_period = tenant_conf
.heatmap_period
.unwrap_or(self.conf.default_tenant_conf.heatmap_period);
if heatmap_period.is_zero() {
None
} else {
Some(heatmap_period)
}
}
pub fn set_new_tenant_config(&self, new_tenant_conf: TenantConfOpt) {
self.tenant_conf.write().unwrap().tenant_conf = new_tenant_conf;
// Don't hold self.timelines.lock() during the notifies.
@@ -3694,6 +3715,7 @@ pub(crate) mod harness {
tenant_conf.evictions_low_residence_duration_metric_threshold,
),
gc_feedback: Some(tenant_conf.gc_feedback),
heatmap_period: Some(tenant_conf.heatmap_period),
}
}
}

View File

@@ -334,6 +334,11 @@ pub struct TenantConf {
#[serde(with = "humantime_serde")]
pub evictions_low_residence_duration_metric_threshold: Duration,
pub gc_feedback: bool,
/// If non-zero, the period between uploads of a heatmap from attached tenants. This
/// may be disabled if a Tenant will not have secondary locations: only secondary
/// locations will use the heatmap uploaded by attached locations.
pub heatmap_period: Duration,
}
/// Same as TenantConf, but this struct preserves the information about
@@ -414,6 +419,11 @@ pub struct TenantConfOpt {
#[serde(skip_serializing_if = "Option::is_none")]
#[serde(default)]
pub gc_feedback: Option<bool>,
#[serde(skip_serializing_if = "Option::is_none")]
#[serde(with = "humantime_serde")]
#[serde(default)]
pub heatmap_period: Option<Duration>,
}
#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)]
@@ -482,6 +492,7 @@ impl TenantConfOpt {
.evictions_low_residence_duration_metric_threshold
.unwrap_or(global_conf.evictions_low_residence_duration_metric_threshold),
gc_feedback: self.gc_feedback.unwrap_or(global_conf.gc_feedback),
heatmap_period: self.heatmap_period.unwrap_or(global_conf.heatmap_period),
}
}
}
@@ -519,6 +530,7 @@ impl Default for TenantConf {
)
.expect("cannot parse default evictions_low_residence_duration_metric_threshold"),
gc_feedback: false,
heatmap_period: Duration::ZERO,
}
}
}

View File

@@ -807,6 +807,12 @@ pub(crate) async fn set_new_tenant_config(
}
impl TenantManager {
/// Convenience function so that anyone with a TenantManager can get at the global configuration, without
/// having to pass it around everywhere as a separate object.
pub(crate) fn get_conf(&self) -> &'static PageServerConf {
self.conf
}
/// Gets the attached tenant from the in-memory data, erroring if it's absent, in secondary mode, or is not fitting to the query.
/// `active_only = true` allows to query only tenants that are ready for operations, erroring on other kinds of tenants.
pub(crate) fn get_attached_tenant_shard(
@@ -1087,6 +1093,20 @@ impl TenantManager {
Ok(())
}
pub(crate) fn get_attached_active_tenant_shards(&self) -> Vec<Arc<Tenant>> {
let locked = self.tenants.read().unwrap();
match &*locked {
TenantsMap::Initializing => Vec::new(),
TenantsMap::Open(map) | TenantsMap::ShuttingDown(map) => map
.values()
.filter_map(|slot| {
slot.get_attached()
.and_then(|t| if t.is_active() { Some(t.clone()) } else { None })
})
.collect(),
}
}
}
#[derive(Debug, thiserror::Error)]

View File

@@ -180,7 +180,7 @@
//! [`Tenant::timeline_init_and_sync`]: super::Tenant::timeline_init_and_sync
//! [`Timeline::load_layer_map`]: super::Timeline::load_layer_map
mod download;
pub(crate) mod download;
pub mod index;
mod upload;
@@ -1604,6 +1604,23 @@ impl RemoteTimelineClient {
}
}
}
pub(crate) fn get_layers_metadata(
&self,
layers: Vec<LayerFileName>,
) -> anyhow::Result<Vec<Option<LayerFileMetadata>>> {
let q = self.upload_queue.lock().unwrap();
let q = match &*q {
UploadQueue::Stopped(_) | UploadQueue::Uninitialized => {
anyhow::bail!("queue is in state {}", q.as_str())
}
UploadQueue::Initialized(inner) => inner,
};
let decorated = layers.into_iter().map(|l| q.latest_files.get(&l).cloned());
Ok(decorated.collect())
}
}
pub fn remote_timelines_path(tenant_shard_id: &TenantShardId) -> RemotePath {
@@ -1659,6 +1676,13 @@ pub fn remote_index_path(
.expect("Failed to construct path")
}
pub const HEATMAP_BASENAME: &str = "heatmap-v1.json";
pub(crate) fn remote_heatmap_path(tenant_shard_id: &TenantShardId) -> RemotePath {
RemotePath::from_string(&format!("tenants/{tenant_shard_id}/{HEATMAP_BASENAME}"))
.expect("Failed to construct path")
}
/// Given the key of an index, parse out the generation part of the name
pub fn parse_remote_index_path(path: RemotePath) -> Option<Generation> {
let file_name = match path.get_path().file_name() {

View File

@@ -0,0 +1,104 @@
pub mod heatmap;
mod heatmap_uploader;
use std::sync::Arc;
use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME};
use self::heatmap_uploader::heatmap_uploader_task;
use super::mgr::TenantManager;
use pageserver_api::shard::TenantShardId;
use remote_storage::GenericRemoteStorage;
use tokio_util::sync::CancellationToken;
use utils::completion::Barrier;
enum UploadCommand {
Upload(TenantShardId),
}
struct CommandRequest<T> {
payload: T,
response_tx: tokio::sync::oneshot::Sender<CommandResponse>,
}
struct CommandResponse {
result: anyhow::Result<()>,
}
/// The SecondaryController is a pseudo-rpc client for administrative control of secondary mode downloads,
/// and heatmap uploads. This is not a hot data path: it's primarily a hook for tests,
/// where we want to immediately upload/download for a particular tenant. In normal operation
/// uploads & downloads are autonomous and not driven by this interface.
pub struct SecondaryController {
upload_req_tx: tokio::sync::mpsc::Sender<CommandRequest<UploadCommand>>,
}
impl SecondaryController {
async fn dispatch<T>(
&self,
queue: &tokio::sync::mpsc::Sender<CommandRequest<T>>,
payload: T,
) -> anyhow::Result<()> {
let (response_tx, response_rx) = tokio::sync::oneshot::channel();
queue
.send(CommandRequest {
payload,
response_tx,
})
.await
.map_err(|_| anyhow::anyhow!("Receiver shut down"))?;
let response = response_rx
.await
.map_err(|_| anyhow::anyhow!("Request dropped"))?;
response.result
}
pub async fn upload_tenant(&self, tenant_shard_id: TenantShardId) -> anyhow::Result<()> {
self.dispatch(&self.upload_req_tx, UploadCommand::Upload(tenant_shard_id))
.await
}
}
pub fn spawn_tasks(
tenant_manager: Arc<TenantManager>,
remote_storage: GenericRemoteStorage,
background_jobs_can_start: Barrier,
cancel: CancellationToken,
) -> SecondaryController {
let (upload_req_tx, upload_req_rx) =
tokio::sync::mpsc::channel::<CommandRequest<UploadCommand>>(16);
task_mgr::spawn(
BACKGROUND_RUNTIME.handle(),
TaskKind::SecondaryUploads,
None,
None,
"heatmap uploads",
false,
async move {
heatmap_uploader_task(
tenant_manager,
remote_storage,
upload_req_rx,
background_jobs_can_start,
cancel,
)
.await
},
);
SecondaryController { upload_req_tx }
}
/// For running with remote storage disabled: a SecondaryController that is connected to nothing.
pub fn null_controller() -> SecondaryController {
let (upload_req_tx, _upload_req_rx) =
tokio::sync::mpsc::channel::<CommandRequest<UploadCommand>>(16);
SecondaryController { upload_req_tx }
}

View File

@@ -0,0 +1,64 @@
use std::time::SystemTime;
use crate::tenant::{
remote_timeline_client::index::IndexLayerMetadata, storage_layer::LayerFileName,
};
use serde::{Deserialize, Serialize};
use serde_with::{serde_as, DisplayFromStr, TimestampSeconds};
use utils::{generation::Generation, id::TimelineId};
#[derive(Serialize, Deserialize)]
pub(super) struct HeatMapTenant {
/// Generation of the attached location that uploaded the heatmap: this is not required
/// for correctness, but acts as a hint to secondary locations in order to detect thrashing
/// in the unlikely event that two attached locations are both uploading conflicting heatmaps.
pub(super) generation: Generation,
pub(super) timelines: Vec<HeatMapTimeline>,
}
#[serde_as]
#[derive(Serialize, Deserialize)]
pub(crate) struct HeatMapTimeline {
#[serde_as(as = "DisplayFromStr")]
pub(super) timeline_id: TimelineId,
pub(super) layers: Vec<HeatMapLayer>,
}
#[serde_as]
#[derive(Serialize, Deserialize)]
pub(crate) struct HeatMapLayer {
pub(super) name: LayerFileName,
pub(super) metadata: IndexLayerMetadata,
#[serde_as(as = "TimestampSeconds<i64>")]
pub(super) access_time: SystemTime,
// TODO: an actual 'heat' score that would let secondary locations prioritize downloading
// the hottest layers, rather than trying to simply mirror whatever layers are on-disk on the primary.
}
impl HeatMapLayer {
pub(crate) fn new(
name: LayerFileName,
metadata: IndexLayerMetadata,
access_time: SystemTime,
) -> Self {
Self {
name,
metadata,
access_time,
}
}
}
impl HeatMapTimeline {
pub(crate) fn new(timeline_id: TimelineId, layers: Vec<HeatMapLayer>) -> Self {
Self {
timeline_id,
layers,
}
}
}

View File

@@ -0,0 +1,582 @@
use std::{
collections::HashMap,
sync::{Arc, Weak},
time::{Duration, Instant},
};
use crate::{
metrics::SECONDARY_MODE,
tenant::{
config::AttachmentMode, mgr::TenantManager, remote_timeline_client::remote_heatmap_path,
secondary::CommandResponse, span::debug_assert_current_span_has_tenant_id, Tenant,
},
};
use md5;
use pageserver_api::shard::TenantShardId;
use remote_storage::GenericRemoteStorage;
use tokio::task::JoinSet;
use tokio_util::sync::CancellationToken;
use tracing::instrument;
use utils::{backoff, completion::Barrier};
use super::{heatmap::HeatMapTenant, CommandRequest, UploadCommand};
/// Period between heatmap uploader walking Tenants to look for work to do.
/// If any tenants have a heatmap upload period lower than this, it will be adjusted
/// downward to match.
const DEFAULT_SCHEDULING_INTERVAL: Duration = Duration::from_millis(60000);
const MIN_SCHEDULING_INTERVAL: Duration = Duration::from_millis(1000);
struct WriteInProgress {
barrier: Barrier,
}
struct UploadPending {
tenant: Arc<Tenant>,
last_digest: Option<md5::Digest>,
}
struct WriteComplete {
tenant_shard_id: TenantShardId,
completed_at: Instant,
digest: Option<md5::Digest>,
next_upload: Option<Instant>,
}
/// The heatmap uploader keeps a little bit of per-tenant state, mainly to remember
/// when we last did a write. We only populate this after doing at least one
/// write for a tenant -- this avoids holding state for tenants that have
/// uploads disabled.
struct UploaderTenantState {
// This Weak only exists to enable culling idle instances of this type
// when the Tenant has been deallocated.
tenant: Weak<Tenant>,
/// Digest of the serialized heatmap that we last successfully uploaded
///
/// md5 is generally a bad hash. We use it because it's convenient for interop with AWS S3's ETag,
/// which is also an md5sum.
last_digest: Option<md5::Digest>,
/// When the last upload attempt completed (may have been successful or failed)
last_upload: Option<Instant>,
/// When should we next do an upload? None means never.
next_upload: Option<Instant>,
}
/// This type is owned by a single task ([`heatmap_uploader_task`]) which runs an event
/// handling loop and mutates it as needed: there are no locks here, because that event loop
/// can hold &mut references to this type throughout.
struct HeatmapUploader {
tenant_manager: Arc<TenantManager>,
remote_storage: GenericRemoteStorage,
cancel: CancellationToken,
tenants: HashMap<TenantShardId, UploaderTenantState>,
/// Tenants with work to do, for which tasks should be spawned as soon as concurrency
/// limits permit it.
tenants_pending: std::collections::VecDeque<UploadPending>,
/// Tenants for which a task in `tasks` has been spawned.
tenants_uploading: HashMap<TenantShardId, WriteInProgress>,
tasks: JoinSet<()>,
/// Channel for our child tasks to send results to: we use a channel for results rather than
/// just getting task results via JoinSet because we need the channel's recv() "sleep until something
/// is available" semantic, rather than JoinSet::join_next()'s "sleep until next thing is available _or_ I'm empty"
/// behavior.
task_result_tx: tokio::sync::mpsc::UnboundedSender<WriteComplete>,
task_result_rx: tokio::sync::mpsc::UnboundedReceiver<WriteComplete>,
concurrent_uploads: usize,
scheduling_interval: Duration,
}
/// The uploader task runs a loop that periodically wakes up and schedules tasks for
/// tenants that require an upload, or handles any commands that have been sent into
/// `command_queue`. No I/O is done in this loop: that all happens in the tasks we
/// spawn.
///
/// Scheduling iterations are somewhat infrequent. However, each one will enqueue
/// all tenants that require an upload, and in between scheduling iterations we will
/// continue to spawn new tasks for pending tenants, as our concurrency limit permits.
///
/// While we take a CancellationToken here, it is subordinate to the CancellationTokens
/// of tenants: i.e. we expect all Tenants to have been shut down before we are shut down, otherwise
/// we might block waiting on a Tenant.
pub(super) async fn heatmap_uploader_task(
tenant_manager: Arc<TenantManager>,
remote_storage: GenericRemoteStorage,
mut command_queue: tokio::sync::mpsc::Receiver<CommandRequest<UploadCommand>>,
background_jobs_can_start: Barrier,
cancel: CancellationToken,
) -> anyhow::Result<()> {
let concurrent_uploads = tenant_manager.get_conf().heatmap_upload_concurrency;
let (result_tx, result_rx) = tokio::sync::mpsc::unbounded_channel();
let mut uploader = HeatmapUploader {
tenant_manager,
remote_storage,
cancel: cancel.clone(),
tasks: JoinSet::new(),
tenants: HashMap::new(),
tenants_pending: std::collections::VecDeque::new(),
tenants_uploading: HashMap::new(),
task_result_tx: result_tx,
task_result_rx: result_rx,
concurrent_uploads,
scheduling_interval: DEFAULT_SCHEDULING_INTERVAL,
};
tracing::info!("Waiting for background_jobs_can start...");
background_jobs_can_start.wait().await;
tracing::info!("background_jobs_can is ready, proceeding.");
while !cancel.is_cancelled() {
// Look for new work: this is relatively expensive because we have to go acquire the lock on
// the tenant manager to retrieve tenants, and then iterate over them to figure out which ones
// require an upload.
uploader.schedule_iteration().await?;
// Between scheduling iterations, we will:
// - Drain any complete tasks and spawn pending tasks
// - Handle incoming administrative commands
// - Check our cancellation token
let next_scheduling_iteration = Instant::now()
.checked_add(uploader.scheduling_interval)
.unwrap_or_else(|| {
tracing::warn!(
"Scheduling interval invalid ({}s), running immediately!",
uploader.scheduling_interval.as_secs_f64()
);
Instant::now()
});
loop {
tokio::select! {
_ = cancel.cancelled() => {
// We do not simply drop the JoinSet, in order to have an orderly shutdown without cancellation.
tracing::info!("Heatmap uploader joining tasks");
while let Some(_r) = uploader.tasks.join_next().await {};
tracing::info!("Heatmap uploader terminating");
break;
},
_ = tokio::time::sleep(next_scheduling_iteration.duration_since(Instant::now())) => {
tracing::debug!("heatmap_uploader_task: woke for scheduling interval");
break;},
cmd = command_queue.recv() => {
tracing::debug!("heatmap_uploader_task: woke for command queue");
let cmd = match cmd {
Some(c) =>c,
None => {
// SecondaryController was destroyed, and this has raced with
// our CancellationToken
tracing::info!("Heatmap uploader terminating");
cancel.cancel();
break;
}
};
let CommandRequest{
response_tx,
payload
} = cmd;
uploader.handle_command(payload, response_tx);
},
_ = uploader.process_next_completion() => {
if !cancel.is_cancelled() {
uploader.spawn_pending();
}
}
}
}
}
Ok(())
}
impl HeatmapUploader {
/// Periodic execution phase: inspect all attached tenants and schedule any work they require.
async fn schedule_iteration(&mut self) -> anyhow::Result<()> {
// Cull any entries in self.tenants whose Arc<Tenant> is gone
self.tenants
.retain(|_k, v| v.tenant.upgrade().is_some() && v.next_upload.is_some());
// The priority order of previously scheduled work may be invalidated by current state: drop
// all pending work (it will be re-scheduled if still needed)
self.tenants_pending.clear();
// Used a fixed 'now' through the following loop, for efficiency and fairness.
let now = Instant::now();
// While iterating over the potentially-long list of tenants, we will periodically yield
// to avoid blocking executor.
const YIELD_ITERATIONS: usize = 1000;
// Iterate over tenants looking for work to do.
let tenants = self.tenant_manager.get_attached_active_tenant_shards();
for (i, tenant) in tenants.into_iter().enumerate() {
// Process is shutting down, drop out
if self.cancel.is_cancelled() {
return Ok(());
}
// Skip tenants that already have a write in flight
if self
.tenants_uploading
.contains_key(tenant.get_tenant_shard_id())
{
continue;
}
self.maybe_schedule_upload(&now, tenant);
if i + 1 % YIELD_ITERATIONS == 0 {
tokio::task::yield_now().await;
}
}
// Spawn tasks for as many of our pending tenants as we can.
self.spawn_pending();
Ok(())
}
///
/// Cancellation: this method is cancel-safe.
async fn process_next_completion(&mut self) {
match self.task_result_rx.recv().await {
Some(r) => {
self.on_completion(r);
}
None => {
unreachable!("Result sender is stored on Self");
}
}
}
/// The 'maybe' refers to the tenant's state: whether it is configured
/// for heatmap uploads at all, and whether sufficient time has passed
/// since the last upload.
fn maybe_schedule_upload(&mut self, now: &Instant, tenant: Arc<Tenant>) {
match tenant.get_heatmap_period() {
None => {
// Heatmaps are disabled for this tenant
return;
}
Some(period) => {
// If any tenant has asked for uploads more frequent than our scheduling interval,
// reduce it to match so that we can keep up. This is mainly useful in testing, where
// we may set rather short intervals.
if period < self.scheduling_interval {
self.scheduling_interval = std::cmp::max(period, MIN_SCHEDULING_INTERVAL);
}
}
}
// Stale attachments do not upload anything: if we are in this state, there is probably some
// other attachment in mode Single or Multi running on another pageserver, and we don't
// want to thrash and overwrite their heatmap uploads.
if tenant.get_attach_mode() == AttachmentMode::Stale {
return;
}
// Create an entry in self.tenants if one doesn't already exist: this will later be updated
// with the completion time in on_completion.
let state = self
.tenants
.entry(*tenant.get_tenant_shard_id())
.or_insert_with(|| UploaderTenantState {
tenant: Arc::downgrade(&tenant),
last_upload: None,
next_upload: Some(Instant::now()),
last_digest: None,
});
// Decline to do the upload if insufficient time has passed
if state.next_upload.map(|nu| &nu > now).unwrap_or(false) {
return;
}
let last_digest = state.last_digest;
self.tenants_pending.push_back(UploadPending {
tenant,
last_digest,
})
}
fn spawn_pending(&mut self) {
while !self.tenants_pending.is_empty()
&& self.tenants_uploading.len() < self.concurrent_uploads
{
// unwrap: loop condition includes !is_empty()
let pending = self.tenants_pending.pop_front().unwrap();
self.spawn_upload(pending.tenant, pending.last_digest);
}
}
fn spawn_upload(&mut self, tenant: Arc<Tenant>, last_digest: Option<md5::Digest>) {
let remote_storage = self.remote_storage.clone();
let tenant_shard_id = *tenant.get_tenant_shard_id();
let (completion, barrier) = utils::completion::channel();
let result_tx = self.task_result_tx.clone();
self.tasks.spawn(async move {
// Guard for the barrier in [`WriteInProgress`]
let _completion = completion;
let started_at = Instant::now();
let digest = match upload_tenant_heatmap(remote_storage, &tenant, last_digest).await {
Ok(UploadHeatmapOutcome::Uploaded(digest)) => {
let duration = Instant::now().duration_since(started_at);
SECONDARY_MODE
.upload_heatmap_duration
.observe(duration.as_secs_f64());
SECONDARY_MODE.upload_heatmap.inc();
Some(digest)
}
Ok(UploadHeatmapOutcome::NoChange | UploadHeatmapOutcome::Skipped) => last_digest,
Err(UploadHeatmapError::Upload(e)) => {
tracing::warn!(
"Failed to upload heatmap for tenant {}: {e:#}",
tenant.get_tenant_shard_id(),
);
let duration = Instant::now().duration_since(started_at);
SECONDARY_MODE
.upload_heatmap_duration
.observe(duration.as_secs_f64());
SECONDARY_MODE.upload_heatmap_errors.inc();
last_digest
}
Err(UploadHeatmapError::Cancelled) => {
tracing::info!("Cancelled heatmap upload, shutting down");
last_digest
}
};
let now = Instant::now();
let next_upload = tenant
.get_heatmap_period()
.and_then(|period| now.checked_add(period));
result_tx
.send(WriteComplete {
tenant_shard_id: *tenant.get_tenant_shard_id(),
completed_at: now,
digest,
next_upload,
})
.ok();
});
self.tenants_uploading
.insert(tenant_shard_id, WriteInProgress { barrier });
}
#[instrument(skip_all, fields(tenant_id=%completion.tenant_shard_id.tenant_id, shard_id=%completion.tenant_shard_id.shard_slug()))]
fn on_completion(&mut self, completion: WriteComplete) {
tracing::debug!("Heatmap upload completed");
let WriteComplete {
tenant_shard_id,
completed_at,
digest,
next_upload,
} = completion;
self.tenants_uploading.remove(&tenant_shard_id);
use std::collections::hash_map::Entry;
match self.tenants.entry(tenant_shard_id) {
Entry::Vacant(_) => {
// Tenant state was dropped, nothing to update.
}
Entry::Occupied(mut entry) => {
entry.get_mut().last_upload = Some(completed_at);
entry.get_mut().last_digest = digest;
entry.get_mut().next_upload = next_upload
}
}
}
fn handle_command(
&mut self,
command: UploadCommand,
response_tx: tokio::sync::oneshot::Sender<CommandResponse>,
) {
match command {
UploadCommand::Upload(tenant_shard_id) => {
// If an upload was ongoing for this tenant, let it finish first.
let barrier = if let Some(writing_state) =
self.tenants_uploading.get(&tenant_shard_id)
{
tracing::info!(
tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(),
"Waiting for heatmap write to complete");
writing_state.barrier.clone()
} else {
// Spawn the upload then immediately wait for it. This will block processing of other commands and
// starting of other background work.
tracing::info!(
tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(),
"Starting heatmap write on command");
let tenant = match self
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id, true)
{
Ok(t) => t,
Err(e) => {
// Drop result of send: we don't care if caller dropped their receiver
drop(response_tx.send(CommandResponse {
result: Err(e.into()),
}));
return;
}
};
self.spawn_upload(tenant, None);
let writing_state = self
.tenants_uploading
.get(&tenant_shard_id)
.expect("We just inserted this");
tracing::info!(
tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(),
"Waiting for heatmap upload to complete");
writing_state.barrier.clone()
};
// This task does no I/O: it only listens for a barrier's completion and then
// sends to the command response channel. It is therefore safe to spawn this without
// any gates/task_mgr hooks.
tokio::task::spawn(async move {
barrier.wait().await;
tracing::info!(
tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(),
"Heatmap upload complete");
// Drop result of send: we don't care if caller dropped their receiver
drop(response_tx.send(CommandResponse { result: Ok(()) }))
});
}
}
}
}
enum UploadHeatmapOutcome {
/// We successfully wrote to remote storage, with this digest.
Uploaded(md5::Digest),
/// We did not upload because the heatmap digest was unchanged since the last upload
NoChange,
/// We skipped the upload for some reason, such as tenant/timeline not ready
Skipped,
}
#[derive(thiserror::Error, Debug)]
enum UploadHeatmapError {
#[error("Cancelled")]
Cancelled,
#[error(transparent)]
Upload(#[from] anyhow::Error),
}
/// The inner upload operation. This will skip if `last_digest` is Some and matches the digest
/// of the object we would have uploaded.
#[instrument(skip_all, fields(tenant_id = %tenant.get_tenant_shard_id().tenant_id, shard_id = %tenant.get_tenant_shard_id().shard_slug()))]
async fn upload_tenant_heatmap(
remote_storage: GenericRemoteStorage,
tenant: &Arc<Tenant>,
last_digest: Option<md5::Digest>,
) -> Result<UploadHeatmapOutcome, UploadHeatmapError> {
debug_assert_current_span_has_tenant_id();
let generation = tenant.get_generation();
if generation.is_none() {
// We do not expect this: generations were implemented before heatmap uploads. However,
// handle it so that we don't have to make the generation in the heatmap an Option<>
// (Generation::none is not serializable)
tracing::warn!("Skipping heatmap upload for tenant with generation==None");
return Ok(UploadHeatmapOutcome::Skipped);
}
let mut heatmap = HeatMapTenant {
timelines: Vec::new(),
generation,
};
let timelines = tenant.timelines.lock().unwrap().clone();
let tenant_cancel = tenant.cancel.clone();
// Ensure that Tenant::shutdown waits for any upload in flight: this is needed because otherwise
// when we delete a tenant, we might race with an upload in flight and end up leaving a heatmap behind
// in remote storage.
let _guard = match tenant.gate.enter() {
Ok(g) => g,
Err(_) => {
tracing::info!("Skipping heatmap upload for tenant which is shutting down");
return Err(UploadHeatmapError::Cancelled);
}
};
for (timeline_id, timeline) in timelines {
let heatmap_timeline = timeline.generate_heatmap().await;
match heatmap_timeline {
None => {
tracing::debug!(
"Skipping heatmap upload because timeline {timeline_id} is not ready"
);
return Ok(UploadHeatmapOutcome::Skipped);
}
Some(heatmap_timeline) => {
heatmap.timelines.push(heatmap_timeline);
}
}
}
// Serialize the heatmap
let bytes = serde_json::to_vec(&heatmap).map_err(|e| anyhow::anyhow!(e))?;
let size = bytes.len();
// Drop out early if nothing changed since our last upload
let digest = md5::compute(&bytes);
if Some(digest) == last_digest {
return Ok(UploadHeatmapOutcome::NoChange);
}
let path = remote_heatmap_path(tenant.get_tenant_shard_id());
// Write the heatmap.
tracing::debug!("Uploading {size} byte heatmap to {path}");
if let Err(e) = backoff::retry(
|| async {
let bytes = futures::stream::once(futures::future::ready(Ok(bytes::Bytes::from(
bytes.clone(),
))));
remote_storage
.upload_storage_object(bytes, size, &path)
.await
},
|_| false,
3,
u32::MAX,
"Uploading heatmap",
backoff::Cancel::new(tenant_cancel.clone(), || anyhow::anyhow!("Shutting down")),
)
.await
{
if tenant_cancel.is_cancelled() {
return Err(UploadHeatmapError::Cancelled);
} else {
return Err(e.into());
}
}
tracing::info!("Successfully uploaded {size} byte heatmap to {path}");
Ok(UploadHeatmapOutcome::Uploaded(digest))
}

View File

@@ -98,8 +98,9 @@ use self::logical_size::LogicalSize;
use self::walreceiver::{WalReceiver, WalReceiverConf};
use super::config::TenantConf;
use super::remote_timeline_client::index::IndexPart;
use super::remote_timeline_client::index::{IndexLayerMetadata, IndexPart};
use super::remote_timeline_client::RemoteTimelineClient;
use super::secondary::heatmap::{HeatMapLayer, HeatMapTimeline};
use super::{debug_assert_current_span_has_tenant_and_timeline_id, AttachedTenantConf};
#[derive(Debug, PartialEq, Eq, Clone, Copy)]
@@ -2055,6 +2056,55 @@ impl Timeline {
None
}
/// The timeline heatmap is a hint to secondary locations from the primary location,
/// indicating which layers are currently on-disk on the primary.
///
/// None is returned if the Timeline is in a state where uploading a heatmap
/// doesn't make sense, such as shutting down or initializing. The caller
/// should treat this as a cue to simply skip doing any heatmap uploading
/// for this timeline.
pub(crate) async fn generate_heatmap(&self) -> Option<HeatMapTimeline> {
let eviction_info = self.get_local_layers_for_disk_usage_eviction().await;
let remote_client = match &self.remote_client {
Some(c) => c,
None => return None,
};
let layer_file_names = eviction_info
.resident_layers
.iter()
.map(|l| l.layer.layer_desc().filename())
.collect::<Vec<_>>();
let decorated = match remote_client.get_layers_metadata(layer_file_names) {
Ok(d) => d,
Err(_) => {
// Getting metadata only fails on Timeline in bad state.
return None;
}
};
let heatmap_layers = std::iter::zip(
eviction_info.resident_layers.into_iter(),
decorated.into_iter(),
)
.filter_map(|(layer, remote_info)| {
remote_info.map(|remote_info| {
HeatMapLayer::new(
layer.layer.layer_desc().filename(),
IndexLayerMetadata::from(remote_info),
layer.last_activity_ts,
)
})
});
Some(HeatMapTimeline::new(
self.timeline_id,
heatmap_layers.collect(),
))
}
}
type TraversalId = String;

View File

@@ -322,6 +322,10 @@ class PageserverHttpClient(requests.Session):
self.verbose_error(res)
return TenantConfig.from_json(res.json())
def tenant_heatmap_upload(self, tenant_id: TenantId):
res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/heatmap_upload")
self.verbose_error(res)
def set_tenant_config(self, tenant_id: TenantId, config: dict[str, Any]):
assert "tenant_id" not in config.keys()
res = self.put(

View File

@@ -16,6 +16,7 @@ from fixtures.log_helper import log
from fixtures.types import TenantId, TimelineId
TIMELINE_INDEX_PART_FILE_NAME = "index_part.json"
TENANT_HEATMAP_FILE_NAME = "heatmap-v1.json"
@enum.unique
@@ -133,6 +134,13 @@ class LocalFsStorage:
with self.index_path(tenant_id, timeline_id).open("r") as f:
return json.load(f)
def heatmap_path(self, tenant_id: TenantId) -> Path:
return self.tenant_path(tenant_id) / TENANT_HEATMAP_FILE_NAME
def heatmap_content(self, tenant_id):
with self.heatmap_path(tenant_id).open("r") as f:
return json.load(f)
def to_toml_inline_table(self) -> str:
rv = {
"local_path": str(self.root),

View File

@@ -163,6 +163,7 @@ def test_fully_custom_config(positive_env: NeonEnv):
"gc_feedback": True,
"gc_horizon": 23 * (1024 * 1024),
"gc_period": "2h 13m",
"heatmap_period": "10m",
"image_creation_threshold": 7,
"pitr_interval": "1m",
"lagging_wal_timeout": "23m",

View File

@@ -4,7 +4,7 @@ from typing import Any, Dict, Optional
import pytest
from fixtures.log_helper import log
from fixtures.neon_fixtures import NeonEnvBuilder, NeonPageserver
from fixtures.remote_storage import RemoteStorageKind
from fixtures.remote_storage import LocalFsStorage, RemoteStorageKind
from fixtures.types import TenantId, TimelineId
from fixtures.utils import wait_until
from fixtures.workload import Workload
@@ -330,3 +330,46 @@ def test_live_migration(neon_env_builder: NeonEnvBuilder):
workload.churn_rows(64, pageserver_b.id)
workload.validate(pageserver_b.id)
def test_heatmap_uploads(neon_env_builder: NeonEnvBuilder):
"""
Test the sequence of location states that are used in a live migration.
"""
env = neon_env_builder.init_start() # initial_tenant_conf=TENANT_CONF)
assert isinstance(env.pageserver_remote_storage, LocalFsStorage)
tenant_id = env.initial_tenant
timeline_id = env.initial_timeline
# Write some data so that we have some layers
workload = Workload(env, tenant_id, timeline_id)
workload.init(env.pageservers[0].id)
# Write some layers and upload a heatmap
workload.write_rows(256, env.pageservers[0].id)
env.pageserver.http_client().tenant_heatmap_upload(tenant_id)
def validate_heatmap(heatmap):
assert len(heatmap["timelines"]) == 1
assert heatmap["timelines"][0]["timeline_id"] == str(timeline_id)
assert len(heatmap["timelines"][0]["layers"]) > 0
layers = heatmap["timelines"][0]["layers"]
# Each layer appears at most once
assert len(set(layer["name"] for layer in layers)) == len(layers)
# Download and inspect the heatmap that the pageserver uploaded
heatmap_first = env.pageserver_remote_storage.heatmap_content(tenant_id)
log.info(f"Read back heatmap: {heatmap_first}")
validate_heatmap(heatmap_first)
# Do some more I/O to generate more layers
workload.churn_rows(64, env.pageservers[0].id)
env.pageserver.http_client().tenant_heatmap_upload(tenant_id)
# Ensure that another heatmap upload includes the new layers
heatmap_second = env.pageserver_remote_storage.heatmap_content(tenant_id)
log.info(f"Read back heatmap: {heatmap_second}")
assert heatmap_second != heatmap_first
validate_heatmap(heatmap_second)