mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-20 20:02:56 +00:00
Compare commits
6 Commits
improve-id
...
jcsp/relat
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
d004675e9b | ||
|
|
5774578fa7 | ||
|
|
d4d4741485 | ||
|
|
171663be86 | ||
|
|
858e93d91a | ||
|
|
275471a2ba |
@@ -1,17 +1,3 @@
|
||||
# The binaries are really slow, if you compile them in 'dev' mode with the defaults.
|
||||
# Enable some optimizations even in 'dev' mode, to make tests faster. The basic
|
||||
# optimizations enabled by "opt-level=1" don't affect debuggability too much.
|
||||
#
|
||||
# See https://www.reddit.com/r/rust/comments/gvrgca/this_is_a_neat_trick_for_getting_good_runtime/
|
||||
#
|
||||
[profile.dev.package."*"]
|
||||
# Set the default for dependencies in Development mode.
|
||||
opt-level = 3
|
||||
|
||||
[profile.dev]
|
||||
# Turn on a small amount of optimization in Development mode.
|
||||
opt-level = 1
|
||||
|
||||
[build]
|
||||
# This is only present for local builds, as it will be overridden
|
||||
# by the RUSTDOCFLAGS env var in CI.
|
||||
|
||||
@@ -1,4 +1,7 @@
|
||||
use crate::repository::{key_range_size, singleton_range, Key};
|
||||
use crate::{
|
||||
pgdatadir_mapping::{BASEBACKUP_CUT, METADATA_CUT},
|
||||
repository::{key_range_size, singleton_range, Key},
|
||||
};
|
||||
use postgres_ffi::BLCKSZ;
|
||||
use std::ops::Range;
|
||||
|
||||
@@ -22,13 +25,22 @@ impl KeySpace {
|
||||
let target_nblocks = (target_size / BLCKSZ as u64) as usize;
|
||||
|
||||
let mut parts = Vec::new();
|
||||
let mut current_part = Vec::new();
|
||||
let mut current_part: Vec<Range<Key>> = Vec::new();
|
||||
let mut current_part_size: usize = 0;
|
||||
for range in &self.ranges {
|
||||
let last = current_part
|
||||
.last()
|
||||
.map(|r| r.end)
|
||||
.unwrap_or(Key::from_i128(0));
|
||||
let cut_here = (range.start >= METADATA_CUT && last < METADATA_CUT)
|
||||
|| (range.start >= BASEBACKUP_CUT && last < BASEBACKUP_CUT);
|
||||
|
||||
// If appending the next contiguous range in the keyspace to the current
|
||||
// partition would cause it to be too large, start a new partition.
|
||||
let this_size = key_range_size(range) as usize;
|
||||
if current_part_size + this_size > target_nblocks && !current_part.is_empty() {
|
||||
if cut_here
|
||||
|| current_part_size + this_size > target_nblocks && !current_part.is_empty()
|
||||
{
|
||||
parts.push(KeySpace {
|
||||
ranges: current_part,
|
||||
});
|
||||
|
||||
@@ -40,9 +40,6 @@ pub enum StorageTimeOperation {
|
||||
#[strum(serialize = "logical size")]
|
||||
LogicalSize,
|
||||
|
||||
#[strum(serialize = "imitate logical size")]
|
||||
ImitateLogicalSize,
|
||||
|
||||
#[strum(serialize = "load layer map")]
|
||||
LoadLayerMap,
|
||||
|
||||
@@ -1364,7 +1361,6 @@ pub struct TimelineMetrics {
|
||||
pub compact_time_histo: StorageTimeMetrics,
|
||||
pub create_images_time_histo: StorageTimeMetrics,
|
||||
pub logical_size_histo: StorageTimeMetrics,
|
||||
pub imitate_logical_size_histo: StorageTimeMetrics,
|
||||
pub load_layer_map_histo: StorageTimeMetrics,
|
||||
pub garbage_collect_histo: StorageTimeMetrics,
|
||||
pub last_record_gauge: IntGauge,
|
||||
@@ -1393,11 +1389,6 @@ impl TimelineMetrics {
|
||||
StorageTimeMetrics::new(StorageTimeOperation::CreateImages, &tenant_id, &timeline_id);
|
||||
let logical_size_histo =
|
||||
StorageTimeMetrics::new(StorageTimeOperation::LogicalSize, &tenant_id, &timeline_id);
|
||||
let imitate_logical_size_histo = StorageTimeMetrics::new(
|
||||
StorageTimeOperation::ImitateLogicalSize,
|
||||
&tenant_id,
|
||||
&timeline_id,
|
||||
);
|
||||
let load_layer_map_histo =
|
||||
StorageTimeMetrics::new(StorageTimeOperation::LoadLayerMap, &tenant_id, &timeline_id);
|
||||
let garbage_collect_histo =
|
||||
@@ -1430,7 +1421,6 @@ impl TimelineMetrics {
|
||||
compact_time_histo,
|
||||
create_images_time_histo,
|
||||
logical_size_histo,
|
||||
imitate_logical_size_histo,
|
||||
garbage_collect_histo,
|
||||
load_layer_map_histo,
|
||||
last_record_gauge,
|
||||
|
||||
@@ -662,20 +662,21 @@ impl Timeline {
|
||||
ctx: &RequestContext,
|
||||
) -> Result<KeySpace, CollectKeySpaceError> {
|
||||
// Iterate through key ranges, greedily packing them into partitions
|
||||
// This function is responsible for appending keys in order, using implicit
|
||||
// knowledge of how keys are defined.
|
||||
let mut result = KeySpaceAccum::new();
|
||||
|
||||
// The dbdir metadata always exists
|
||||
result.add_key(DBDIR_KEY);
|
||||
|
||||
// Fetch list of database dirs and iterate them
|
||||
let buf = self.get(DBDIR_KEY, lsn, ctx).await?;
|
||||
let dbdir = DbDirectory::des(&buf)?;
|
||||
|
||||
let mut metadata_keys = Vec::new();
|
||||
|
||||
let mut dbs: Vec<(Oid, Oid)> = dbdir.dbdirs.keys().cloned().collect();
|
||||
dbs.sort_unstable();
|
||||
for (spcnode, dbnode) in dbs {
|
||||
result.add_key(relmap_file_key(spcnode, dbnode));
|
||||
result.add_key(rel_dir_to_key(spcnode, dbnode));
|
||||
metadata_keys.push(relmap_file_key(spcnode, dbnode));
|
||||
metadata_keys.push(rel_dir_to_key(spcnode, dbnode));
|
||||
|
||||
let mut rels: Vec<RelTag> = self
|
||||
.list_rels(spcnode, dbnode, lsn, ctx)
|
||||
@@ -689,7 +690,7 @@ impl Timeline {
|
||||
let relsize = buf.get_u32_le();
|
||||
|
||||
result.add_range(rel_block_to_key(rel, 0)..rel_block_to_key(rel, relsize));
|
||||
result.add_key(relsize_key);
|
||||
metadata_keys.push(relsize_key);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -732,6 +733,13 @@ impl Timeline {
|
||||
if self.get(AUX_FILES_KEY, lsn, ctx).await.is_ok() {
|
||||
result.add_key(AUX_FILES_KEY);
|
||||
}
|
||||
|
||||
// The dbdir metadata always exists
|
||||
result.add_key(DBDIR_KEY);
|
||||
for key in metadata_keys {
|
||||
result.add_key(key);
|
||||
}
|
||||
|
||||
Ok(result.to_keyspace())
|
||||
}
|
||||
|
||||
@@ -1474,21 +1482,11 @@ static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; BLCKSZ as usize]);
|
||||
//
|
||||
// Below is a full list of the keyspace allocation:
|
||||
//
|
||||
// DbDir:
|
||||
// 00 00000000 00000000 00000000 00 00000000
|
||||
//
|
||||
// Filenodemap:
|
||||
// 00 SPCNODE DBNODE 00000000 00 00000000
|
||||
//
|
||||
// RelDir:
|
||||
// 00 SPCNODE DBNODE 00000000 00 00000001 (Postgres never uses relfilenode 0)
|
||||
|
||||
//
|
||||
// RelBlock:
|
||||
// 00 SPCNODE DBNODE RELNODE FORK BLKNUM
|
||||
//
|
||||
// RelSize:
|
||||
// 00 SPCNODE DBNODE RELNODE FORK FFFFFFFF
|
||||
//
|
||||
// SlruDir:
|
||||
// 01 kind 00000000 00000000 00 00000000
|
||||
//
|
||||
@@ -1513,11 +1511,31 @@ static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; BLCKSZ as usize]);
|
||||
// AuxFiles:
|
||||
// 03 00000000 00000000 00000000 00 00000002
|
||||
//
|
||||
// DbDir:
|
||||
// 04 00000000 00000000 00000000 00 00000000
|
||||
//
|
||||
// Filenodemap:
|
||||
// 04 SPCNODE DBNODE 00000000 00 00000000
|
||||
//
|
||||
// RelDir:
|
||||
// 04 SPCNODE DBNODE 00000000 00 00000001 (Postgres never uses relfilenode 0)
|
||||
//
|
||||
// RelSize:
|
||||
// 04 SPCNODE DBNODE RELNODE FORK FFFFFFFF
|
||||
|
||||
//-- Section 01: relation data and metadata
|
||||
|
||||
/// Keys above this Key are required to serve a basebackup request
|
||||
pub(crate) const BASEBACKUP_CUT: Key = slru_dir_to_key(SlruKind::Clog);
|
||||
|
||||
/// Keys aboe this Key are needed to make a logical size calculation
|
||||
///
|
||||
/// Ensuring that such keys are stored above the main range of user relation
|
||||
/// blocks enables much more efficient space management.
|
||||
pub(crate) const METADATA_CUT: Key = CONTROLFILE_KEY;
|
||||
|
||||
const DBDIR_KEY: Key = Key {
|
||||
field1: 0x00,
|
||||
field1: 0x04,
|
||||
field2: 0,
|
||||
field3: 0,
|
||||
field4: 0,
|
||||
@@ -1527,14 +1545,14 @@ const DBDIR_KEY: Key = Key {
|
||||
|
||||
fn dbdir_key_range(spcnode: Oid, dbnode: Oid) -> Range<Key> {
|
||||
Key {
|
||||
field1: 0x00,
|
||||
field1: 0x04,
|
||||
field2: spcnode,
|
||||
field3: dbnode,
|
||||
field4: 0,
|
||||
field5: 0,
|
||||
field6: 0,
|
||||
}..Key {
|
||||
field1: 0x00,
|
||||
field1: 0x04,
|
||||
field2: spcnode,
|
||||
field3: dbnode,
|
||||
field4: 0xffffffff,
|
||||
@@ -1545,7 +1563,7 @@ fn dbdir_key_range(spcnode: Oid, dbnode: Oid) -> Range<Key> {
|
||||
|
||||
fn relmap_file_key(spcnode: Oid, dbnode: Oid) -> Key {
|
||||
Key {
|
||||
field1: 0x00,
|
||||
field1: 0x04,
|
||||
field2: spcnode,
|
||||
field3: dbnode,
|
||||
field4: 0,
|
||||
@@ -1556,7 +1574,7 @@ fn relmap_file_key(spcnode: Oid, dbnode: Oid) -> Key {
|
||||
|
||||
fn rel_dir_to_key(spcnode: Oid, dbnode: Oid) -> Key {
|
||||
Key {
|
||||
field1: 0x00,
|
||||
field1: 0x04,
|
||||
field2: spcnode,
|
||||
field3: dbnode,
|
||||
field4: 0,
|
||||
@@ -1578,7 +1596,7 @@ fn rel_block_to_key(rel: RelTag, blknum: BlockNumber) -> Key {
|
||||
|
||||
fn rel_size_to_key(rel: RelTag) -> Key {
|
||||
Key {
|
||||
field1: 0x00,
|
||||
field1: 0x04,
|
||||
field2: rel.spcnode,
|
||||
field3: rel.dbnode,
|
||||
field4: rel.relnode,
|
||||
@@ -1607,7 +1625,7 @@ fn rel_key_range(rel: RelTag) -> Range<Key> {
|
||||
|
||||
//-- Section 02: SLRUs
|
||||
|
||||
fn slru_dir_to_key(kind: SlruKind) -> Key {
|
||||
const fn slru_dir_to_key(kind: SlruKind) -> Key {
|
||||
Key {
|
||||
field1: 0x01,
|
||||
field2: match kind {
|
||||
|
||||
@@ -61,7 +61,6 @@ use self::mgr::TenantsMap;
|
||||
use self::remote_timeline_client::RemoteTimelineClient;
|
||||
use self::timeline::uninit::TimelineUninitMark;
|
||||
use self::timeline::uninit::UninitializedTimeline;
|
||||
use self::timeline::EvictionTaskTenantState;
|
||||
use self::timeline::TimelineResources;
|
||||
use crate::config::PageServerConf;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
@@ -252,8 +251,6 @@ pub struct Tenant {
|
||||
cached_logical_sizes: tokio::sync::Mutex<HashMap<(TimelineId, Lsn), u64>>,
|
||||
cached_synthetic_tenant_size: Arc<AtomicU64>,
|
||||
|
||||
eviction_task_tenant_state: tokio::sync::Mutex<EvictionTaskTenantState>,
|
||||
|
||||
pub(crate) delete_progress: Arc<tokio::sync::Mutex<DeleteTenantFlow>>,
|
||||
|
||||
// Cancellation token fires when we have entered shutdown(). This is a parent of
|
||||
@@ -2367,7 +2364,6 @@ impl Tenant {
|
||||
state,
|
||||
cached_logical_sizes: tokio::sync::Mutex::new(HashMap::new()),
|
||||
cached_synthetic_tenant_size: Arc::new(AtomicU64::new(0)),
|
||||
eviction_task_tenant_state: tokio::sync::Mutex::new(EvictionTaskTenantState::default()),
|
||||
delete_progress: Arc::new(tokio::sync::Mutex::new(DeleteTenantFlow::default())),
|
||||
cancel: CancellationToken::default(),
|
||||
gate: Gate::new(format!("Tenant<{tenant_id}>")),
|
||||
|
||||
@@ -5,7 +5,7 @@ use utils::{
|
||||
lsn::Lsn,
|
||||
};
|
||||
|
||||
use crate::repository::Key;
|
||||
use crate::{pgdatadir_mapping::METADATA_CUT, repository::Key};
|
||||
|
||||
use super::{DeltaFileName, ImageFileName, LayerFileName};
|
||||
|
||||
@@ -49,6 +49,20 @@ impl PersistentLayerDesc {
|
||||
}
|
||||
}
|
||||
|
||||
/// Does this layer consist exclusively of metadata
|
||||
/// content such as dbdir & relation sizes? This is a
|
||||
/// hint that the layer is likely to be small and should
|
||||
/// not be a candidate for eviction under normal circumstances.
|
||||
pub fn is_metadata_pages(&self) -> bool {
|
||||
self.key_range.start >= METADATA_CUT
|
||||
}
|
||||
|
||||
/// Does this layer consist exclusively of content
|
||||
/// required to serve a basebackup request?
|
||||
pub fn is_basebackup_pages(&self) -> bool {
|
||||
self.key_range.start >= METADATA_CUT
|
||||
}
|
||||
|
||||
pub fn short_id(&self) -> impl Display {
|
||||
self.filename()
|
||||
}
|
||||
|
||||
@@ -81,8 +81,6 @@ use crate::task_mgr::TaskKind;
|
||||
use crate::ZERO_PAGE;
|
||||
|
||||
use self::delete::DeleteTimelineFlow;
|
||||
pub(super) use self::eviction_task::EvictionTaskTenantState;
|
||||
use self::eviction_task::EvictionTaskTimelineState;
|
||||
use self::layer_manager::LayerManager;
|
||||
use self::logical_size::LogicalSize;
|
||||
use self::walreceiver::{WalReceiver, WalReceiverConf};
|
||||
@@ -298,8 +296,6 @@ pub struct Timeline {
|
||||
/// timeline is being deleted. If 'true', the timeline has already been deleted.
|
||||
pub delete_progress: Arc<tokio::sync::Mutex<DeleteTimelineFlow>>,
|
||||
|
||||
eviction_task_timeline_state: tokio::sync::Mutex<EvictionTaskTimelineState>,
|
||||
|
||||
/// Barrier to wait before doing initial logical size calculation. Used only during startup.
|
||||
initial_logical_size_can_start: Option<completion::Barrier>,
|
||||
|
||||
@@ -433,7 +429,6 @@ impl std::fmt::Display for PageReconstructError {
|
||||
pub enum LogicalSizeCalculationCause {
|
||||
Initial,
|
||||
ConsumptionMetricsSyntheticSize,
|
||||
EvictionTaskImitation,
|
||||
TenantSizeHandler,
|
||||
}
|
||||
|
||||
@@ -1442,9 +1437,6 @@ impl Timeline {
|
||||
|
||||
state,
|
||||
|
||||
eviction_task_timeline_state: tokio::sync::Mutex::new(
|
||||
EvictionTaskTimelineState::default(),
|
||||
),
|
||||
delete_progress: Arc::new(tokio::sync::Mutex::new(DeleteTimelineFlow::default())),
|
||||
|
||||
initial_logical_size_can_start,
|
||||
@@ -1967,9 +1959,6 @@ impl Timeline {
|
||||
LogicalSizeCalculationCause::Initial
|
||||
| LogicalSizeCalculationCause::ConsumptionMetricsSyntheticSize
|
||||
| LogicalSizeCalculationCause::TenantSizeHandler => &self.metrics.logical_size_histo,
|
||||
LogicalSizeCalculationCause::EvictionTaskImitation => {
|
||||
&self.metrics.imitate_logical_size_histo
|
||||
}
|
||||
};
|
||||
let timer = storage_time_metrics.start_timer();
|
||||
let logical_size = self
|
||||
@@ -2746,18 +2735,18 @@ impl Timeline {
|
||||
partition_size: u64,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<(KeyPartitioning, Lsn)> {
|
||||
{
|
||||
let partitioning_guard = self.partitioning.lock().unwrap();
|
||||
let distance = lsn.0 - partitioning_guard.1 .0;
|
||||
if partitioning_guard.1 != Lsn(0) && distance <= self.repartition_threshold {
|
||||
debug!(
|
||||
distance,
|
||||
threshold = self.repartition_threshold,
|
||||
"no repartitioning needed"
|
||||
);
|
||||
return Ok((partitioning_guard.0.clone(), partitioning_guard.1));
|
||||
}
|
||||
}
|
||||
// {
|
||||
// let partitioning_guard = self.partitioning.lock().unwrap();
|
||||
// let distance = lsn.0 - partitioning_guard.1 .0;
|
||||
// if partitioning_guard.1 != Lsn(0) && distance <= self.repartition_threshold {
|
||||
// debug!(
|
||||
// distance,
|
||||
// threshold = self.repartition_threshold,
|
||||
// "no repartitioning needed"
|
||||
// );
|
||||
// return Ok((partitioning_guard.0.clone(), partitioning_guard.1));
|
||||
// }
|
||||
// }
|
||||
let keyspace = self.collect_keyspace(lsn, ctx).await?;
|
||||
let partitioning = keyspace.partition(partition_size);
|
||||
|
||||
@@ -4285,6 +4274,11 @@ impl Timeline {
|
||||
let file_size = l.file_size();
|
||||
max_layer_size = max_layer_size.map_or(Some(file_size), |m| Some(m.max(file_size)));
|
||||
|
||||
// Don't evict small layers required to serve a basebackup
|
||||
if l.is_basebackup_pages() {
|
||||
continue;
|
||||
}
|
||||
|
||||
let l = guard.get_from_desc(&l);
|
||||
|
||||
let l = match l.keep_resident().await {
|
||||
|
||||
@@ -14,7 +14,6 @@
|
||||
//!
|
||||
//! See write-up on restart on-demand download spike: <https://gist.github.com/problame/2265bf7b8dc398be834abfead36c76b5>
|
||||
use std::{
|
||||
collections::HashMap,
|
||||
ops::ControlFlow,
|
||||
sync::Arc,
|
||||
time::{Duration, SystemTime},
|
||||
@@ -22,17 +21,15 @@ use std::{
|
||||
|
||||
use tokio::time::Instant;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use tracing::{debug, error, info, info_span, instrument, warn, Instrument};
|
||||
use tracing::{debug, error, info, instrument, warn};
|
||||
|
||||
use crate::{
|
||||
context::{DownloadBehavior, RequestContext},
|
||||
pgdatadir_mapping::CollectKeySpaceError,
|
||||
task_mgr::{self, TaskKind, BACKGROUND_RUNTIME},
|
||||
tenant::{
|
||||
config::{EvictionPolicy, EvictionPolicyLayerAccessThreshold},
|
||||
tasks::{BackgroundLoopKind, RateLimitError},
|
||||
timeline::EvictionError,
|
||||
LogicalSizeCalculationCause, Tenant,
|
||||
},
|
||||
};
|
||||
|
||||
@@ -40,16 +37,6 @@ use utils::completion;
|
||||
|
||||
use super::Timeline;
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct EvictionTaskTimelineState {
|
||||
last_layer_access_imitation: Option<tokio::time::Instant>,
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct EvictionTaskTenantState {
|
||||
last_layer_access_imitation: Option<Instant>,
|
||||
}
|
||||
|
||||
impl Timeline {
|
||||
pub(super) fn launch_eviction_task(
|
||||
self: &Arc<Self>,
|
||||
@@ -178,7 +165,6 @@ impl Timeline {
|
||||
// that were accessed to compute the value in the first place.
|
||||
// 3. Invalidate the caches at a period of < p.threshold/2, so that the values
|
||||
// get re-computed from layers, thereby counting towards layer access stats.
|
||||
// 4. Make the eviction task imitate the layer accesses that typically hit caches.
|
||||
//
|
||||
// We follow approach (4) here because in Neon prod deployment:
|
||||
// - page cache is quite small => high churn => low hit rate
|
||||
@@ -190,10 +176,6 @@ impl Timeline {
|
||||
//
|
||||
// We should probably move to persistent caches in the future, or avoid
|
||||
// having inactive tenants attached to pageserver in the first place.
|
||||
match self.imitate_layer_accesses(p, cancel, ctx).await {
|
||||
ControlFlow::Break(()) => return ControlFlow::Break(()),
|
||||
ControlFlow::Continue(()) => (),
|
||||
}
|
||||
|
||||
#[allow(dead_code)]
|
||||
#[derive(Debug, Default)]
|
||||
@@ -215,6 +197,11 @@ impl Timeline {
|
||||
let layers = guard.layer_map();
|
||||
let mut candidates = Vec::new();
|
||||
for hist_layer in layers.iter_historic_layers() {
|
||||
// Don't evict the small layers needed to serve a basebackup request.
|
||||
if hist_layer.is_basebackup_pages() {
|
||||
continue;
|
||||
}
|
||||
|
||||
let hist_layer = guard.get_from_desc(&hist_layer);
|
||||
|
||||
// guard against eviction while we inspect it; it might be that eviction_task and
|
||||
@@ -310,170 +297,4 @@ impl Timeline {
|
||||
}
|
||||
ControlFlow::Continue(())
|
||||
}
|
||||
|
||||
#[instrument(skip_all)]
|
||||
async fn imitate_layer_accesses(
|
||||
&self,
|
||||
p: &EvictionPolicyLayerAccessThreshold,
|
||||
cancel: &CancellationToken,
|
||||
ctx: &RequestContext,
|
||||
) -> ControlFlow<()> {
|
||||
let mut state = self.eviction_task_timeline_state.lock().await;
|
||||
|
||||
// Only do the imitate_layer accesses approximately as often as the threshold. A little
|
||||
// more frequently, to avoid this period racing with the threshold/period-th eviction iteration.
|
||||
let inter_imitate_period = p.threshold.checked_sub(p.period).unwrap_or(p.threshold);
|
||||
|
||||
match state.last_layer_access_imitation {
|
||||
Some(ts) if ts.elapsed() < inter_imitate_period => { /* no need to run */ }
|
||||
_ => {
|
||||
self.imitate_timeline_cached_layer_accesses(ctx).await;
|
||||
state.last_layer_access_imitation = Some(tokio::time::Instant::now())
|
||||
}
|
||||
}
|
||||
drop(state);
|
||||
|
||||
if cancel.is_cancelled() {
|
||||
return ControlFlow::Break(());
|
||||
}
|
||||
|
||||
// This task is timeline-scoped, but the synthetic size calculation is tenant-scoped.
|
||||
// Make one of the tenant's timelines draw the short straw and run the calculation.
|
||||
// The others wait until the calculation is done so that they take into account the
|
||||
// imitated accesses that the winner made.
|
||||
let tenant = match crate::tenant::mgr::get_tenant(self.tenant_id, true) {
|
||||
Ok(t) => t,
|
||||
Err(_) => {
|
||||
return ControlFlow::Break(());
|
||||
}
|
||||
};
|
||||
let mut state = tenant.eviction_task_tenant_state.lock().await;
|
||||
match state.last_layer_access_imitation {
|
||||
Some(ts) if ts.elapsed() < inter_imitate_period => { /* no need to run */ }
|
||||
_ => {
|
||||
self.imitate_synthetic_size_calculation_worker(&tenant, ctx, cancel)
|
||||
.await;
|
||||
state.last_layer_access_imitation = Some(tokio::time::Instant::now());
|
||||
}
|
||||
}
|
||||
drop(state);
|
||||
|
||||
if cancel.is_cancelled() {
|
||||
return ControlFlow::Break(());
|
||||
}
|
||||
|
||||
ControlFlow::Continue(())
|
||||
}
|
||||
|
||||
/// Recompute the values which would cause on-demand downloads during restart.
|
||||
#[instrument(skip_all)]
|
||||
async fn imitate_timeline_cached_layer_accesses(&self, ctx: &RequestContext) {
|
||||
let lsn = self.get_last_record_lsn();
|
||||
|
||||
// imitiate on-restart initial logical size
|
||||
let size = self
|
||||
.calculate_logical_size(lsn, LogicalSizeCalculationCause::EvictionTaskImitation, ctx)
|
||||
.instrument(info_span!("calculate_logical_size"))
|
||||
.await;
|
||||
|
||||
match &size {
|
||||
Ok(_size) => {
|
||||
// good, don't log it to avoid confusion
|
||||
}
|
||||
Err(_) => {
|
||||
// we have known issues for which we already log this on consumption metrics,
|
||||
// gc, and compaction. leave logging out for now.
|
||||
//
|
||||
// https://github.com/neondatabase/neon/issues/2539
|
||||
}
|
||||
}
|
||||
|
||||
// imitiate repartiting on first compactation
|
||||
if let Err(e) = self
|
||||
.collect_keyspace(lsn, ctx)
|
||||
.instrument(info_span!("collect_keyspace"))
|
||||
.await
|
||||
{
|
||||
// if this failed, we probably failed logical size because these use the same keys
|
||||
if size.is_err() {
|
||||
// ignore, see above comment
|
||||
} else {
|
||||
match e {
|
||||
CollectKeySpaceError::Cancelled => {
|
||||
// Shutting down, ignore
|
||||
}
|
||||
err => {
|
||||
warn!(
|
||||
"failed to collect keyspace but succeeded in calculating logical size: {err:#}"
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Imitate the synthetic size calculation done by the consumption_metrics module.
|
||||
#[instrument(skip_all)]
|
||||
async fn imitate_synthetic_size_calculation_worker(
|
||||
&self,
|
||||
tenant: &Arc<Tenant>,
|
||||
ctx: &RequestContext,
|
||||
cancel: &CancellationToken,
|
||||
) {
|
||||
if self.conf.metric_collection_endpoint.is_none() {
|
||||
// We don't start the consumption metrics task if this is not set in the config.
|
||||
// So, no need to imitate the accesses in that case.
|
||||
return;
|
||||
}
|
||||
|
||||
// The consumption metrics are collected on a per-tenant basis, by a single
|
||||
// global background loop.
|
||||
// It limits the number of synthetic size calculations using the global
|
||||
// `concurrent_tenant_size_logical_size_queries` semaphore to not overload
|
||||
// the pageserver. (size calculation is somewhat expensive in terms of CPU and IOs).
|
||||
//
|
||||
// If we used that same semaphore here, then we'd compete for the
|
||||
// same permits, which may impact timeliness of consumption metrics.
|
||||
// That is a no-go, as consumption metrics are much more important
|
||||
// than what we do here.
|
||||
//
|
||||
// So, we have a separate semaphore, initialized to the same
|
||||
// number of permits as the `concurrent_tenant_size_logical_size_queries`.
|
||||
// In the worst, we would have twice the amount of concurrenct size calculations.
|
||||
// But in practice, the `p.threshold` >> `consumption metric interval`, and
|
||||
// we spread out the eviction task using `random_init_delay`.
|
||||
// So, the chance of the worst case is quite low in practice.
|
||||
// It runs as a per-tenant task, but the eviction_task.rs is per-timeline.
|
||||
// So, we must coordinate with other with other eviction tasks of this tenant.
|
||||
let limit = self
|
||||
.conf
|
||||
.eviction_task_immitated_concurrent_logical_size_queries
|
||||
.inner();
|
||||
|
||||
let mut throwaway_cache = HashMap::new();
|
||||
let gather = crate::tenant::size::gather_inputs(
|
||||
tenant,
|
||||
limit,
|
||||
None,
|
||||
&mut throwaway_cache,
|
||||
LogicalSizeCalculationCause::EvictionTaskImitation,
|
||||
ctx,
|
||||
)
|
||||
.instrument(info_span!("gather_inputs"));
|
||||
|
||||
tokio::select! {
|
||||
_ = cancel.cancelled() => {}
|
||||
gather_result = gather => {
|
||||
match gather_result {
|
||||
Ok(_) => {},
|
||||
Err(e) => {
|
||||
// We don't care about the result, but, if it failed, we should log it,
|
||||
// since consumption metric might be hitting the cached value and
|
||||
// thus not encountering this error.
|
||||
warn!("failed to imitate synthetic size calculation accesses: {e:#}")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user