From 6a53b8fac67f8a5bdf1e2c6f11b759a6e51652ea Mon Sep 17 00:00:00 2001 From: Heikki Linnakangas Date: Thu, 12 Jan 2023 17:26:49 +0200 Subject: [PATCH] Add placeholders for RequestContext and friends. This commit adds the 'ctx' parameter to all the functions that will need an active context. However, you can just create new contexts on the fly, there is no cross-checks that the tenant/timeline is still in active state. You can simply call Tenant::get_context or Timeline::get_context, and they always succee. In the next commit, we will change the functions for constructing contexts, so that you cannot create a new TenantRequestContext if the tenant is being stopped (and similarly for TimelineRequestContext). This commit isn't useful on its own, but splitting these fairly mechanical changes helps to make the next commit smaller, and thus easier to review. Because the contexts are merely passed through places, and not actually used for anything, this introduces a lot of "unused variable" warnings. They will go away with the next commit. --- pageserver/src/basebackup.rs | 52 +- pageserver/src/consumption_metrics.rs | 4 +- pageserver/src/context.rs | 12 + pageserver/src/http/routes.rs | 74 ++- pageserver/src/import_datadir.rs | 51 +- pageserver/src/lib.rs | 1 + pageserver/src/page_service.rs | 145 ++++-- pageserver/src/pgdatadir_mapping.rs | 181 ++++--- pageserver/src/tenant.rs | 468 +++++++++++------- pageserver/src/tenant/mgr.rs | 17 +- pageserver/src/tenant/size.rs | 26 +- pageserver/src/tenant/tasks.rs | 8 +- pageserver/src/tenant/timeline.rs | 127 +++-- pageserver/src/walingest.rs | 419 ++++++++++------ .../src/walreceiver/connection_manager.rs | 23 +- .../src/walreceiver/walreceiver_connection.rs | 9 +- 16 files changed, 1079 insertions(+), 538 deletions(-) create mode 100644 pageserver/src/context.rs diff --git a/pageserver/src/basebackup.rs b/pageserver/src/basebackup.rs index f1d92ac36b..5f16692b56 100644 --- a/pageserver/src/basebackup.rs +++ b/pageserver/src/basebackup.rs @@ -27,7 +27,7 @@ use tracing::*; /// use tokio_tar::{Builder, EntryType, Header}; -use crate::tenant::Timeline; +use crate::tenant::{Timeline, TimelineRequestContext}; use pageserver_api::reltag::{RelTag, SlruKind}; use postgres_ffi::pg_constants::{DEFAULTTABLESPACE_OID, GLOBALTABLESPACE_OID}; @@ -52,6 +52,7 @@ pub async fn send_basebackup_tarball<'a, W>( req_lsn: Option, prev_lsn: Option, full_backup: bool, + ctx: &'a TimelineRequestContext, ) -> anyhow::Result<()> where W: AsyncWrite + Send + Sync + Unpin, @@ -110,6 +111,7 @@ where lsn: backup_lsn, prev_record_lsn: prev_lsn, full_backup, + ctx, }; basebackup .send_tarball() @@ -129,6 +131,7 @@ where lsn: Lsn, prev_record_lsn: Lsn, full_backup: bool, + ctx: &'a TimelineRequestContext, } impl<'a, W> Basebackup<'a, W> @@ -171,23 +174,37 @@ where SlruKind::MultiXactOffsets, SlruKind::MultiXactMembers, ] { - for segno in self.timeline.list_slru_segments(kind, self.lsn).await? { + for segno in self + .timeline + .list_slru_segments(kind, self.lsn, self.ctx) + .await? + { self.add_slru_segment(kind, segno).await?; } } // Create tablespace directories - for ((spcnode, dbnode), has_relmap_file) in self.timeline.list_dbdirs(self.lsn).await? { + for ((spcnode, dbnode), has_relmap_file) in + self.timeline.list_dbdirs(self.lsn, self.ctx).await? + { self.add_dbdir(spcnode, dbnode, has_relmap_file).await?; // Gather and send relational files in each database if full backup is requested. if self.full_backup { - for rel in self.timeline.list_rels(spcnode, dbnode, self.lsn).await? { + for rel in self + .timeline + .list_rels(spcnode, dbnode, self.lsn, self.ctx) + .await? + { self.add_rel(rel).await?; } } } - for xid in self.timeline.list_twophase_files(self.lsn).await? { + for xid in self + .timeline + .list_twophase_files(self.lsn, self.ctx) + .await? + { self.add_twophase_file(xid).await?; } @@ -203,7 +220,10 @@ where } async fn add_rel(&mut self, tag: RelTag) -> anyhow::Result<()> { - let nblocks = self.timeline.get_rel_size(tag, self.lsn, false).await?; + let nblocks = self + .timeline + .get_rel_size(tag, self.lsn, false, self.ctx) + .await?; // If the relation is empty, create an empty file if nblocks == 0 { @@ -218,12 +238,11 @@ where let mut seg = 0; while startblk < nblocks { let endblk = std::cmp::min(startblk + RELSEG_SIZE, nblocks); - let mut segment_data: Vec = vec![]; for blknum in startblk..endblk { let img = self .timeline - .get_rel_page_at_lsn(tag, blknum, self.lsn, false) + .get_rel_page_at_lsn(tag, blknum, self.lsn, false, self.ctx) .await?; segment_data.extend_from_slice(&img[..]); } @@ -245,14 +264,14 @@ where async fn add_slru_segment(&mut self, slru: SlruKind, segno: u32) -> anyhow::Result<()> { let nblocks = self .timeline - .get_slru_segment_size(slru, segno, self.lsn) + .get_slru_segment_size(slru, segno, self.lsn, self.ctx) .await?; let mut slru_buf: Vec = Vec::with_capacity(nblocks as usize * BLCKSZ as usize); for blknum in 0..nblocks { let img = self .timeline - .get_slru_page_at_lsn(slru, segno, blknum, self.lsn) + .get_slru_page_at_lsn(slru, segno, blknum, self.lsn, self.ctx) .await?; if slru == SlruKind::Clog { @@ -287,7 +306,7 @@ where let relmap_img = if has_relmap_file { let img = self .timeline - .get_relmap_file(spcnode, dbnode, self.lsn) + .get_relmap_file(spcnode, dbnode, self.lsn, self.ctx) .await?; ensure!(img.len() == 512); Some(img) @@ -323,7 +342,7 @@ where if !has_relmap_file && self .timeline - .list_rels(spcnode, dbnode, self.lsn) + .list_rels(spcnode, dbnode, self.lsn, self.ctx) .await? .is_empty() { @@ -356,7 +375,10 @@ where // Extract twophase state files // async fn add_twophase_file(&mut self, xid: TransactionId) -> anyhow::Result<()> { - let img = self.timeline.get_twophase_file(xid, self.lsn).await?; + let img = self + .timeline + .get_twophase_file(xid, self.lsn, self.ctx) + .await?; let mut buf = BytesMut::new(); buf.extend_from_slice(&img[..]); @@ -394,12 +416,12 @@ where let checkpoint_bytes = self .timeline - .get_checkpoint(self.lsn) + .get_checkpoint(self.lsn, self.ctx) .await .context("failed to get checkpoint bytes")?; let pg_control_bytes = self .timeline - .get_control_file(self.lsn) + .get_control_file(self.lsn, self.ctx) .await .context("failed get control bytes")?; diff --git a/pageserver/src/consumption_metrics.rs b/pageserver/src/consumption_metrics.rs index c411a9e025..ffecc08385 100644 --- a/pageserver/src/consumption_metrics.rs +++ b/pageserver/src/consumption_metrics.rs @@ -197,6 +197,7 @@ pub async fn collect_metrics_task( // iterate through list of timelines in tenant for timeline in tenant.list_timelines().iter() { // collect per-timeline metrics only for active timelines + let timeline_ctx = timeline.get_context(); if timeline.is_active() { let timeline_written_size = u64::from(timeline.get_last_record_lsn()); @@ -209,7 +210,8 @@ pub async fn collect_metrics_task( timeline_written_size, )); - let (timeline_logical_size, is_exact) = timeline.get_current_logical_size()?; + let (timeline_logical_size, is_exact) = + timeline.get_current_logical_size(&timeline_ctx)?; // Only send timeline logical size when it is fully calculated. if is_exact { current_metrics.push(( diff --git a/pageserver/src/context.rs b/pageserver/src/context.rs new file mode 100644 index 0000000000..912ce7db5d --- /dev/null +++ b/pageserver/src/context.rs @@ -0,0 +1,12 @@ +//! Most async functions throughout the pageserver take a `ctx: &RequestContext` +//! argument. Currently, it's just a placeholder, but in upcoming commit, it +//! will be used for cancellation, and to ensure that a Tenant or Timeline isn't +//! removed while there are still tasks operating on it. + +pub struct RequestContext {} + +impl RequestContext { + pub fn new() -> Self { + RequestContext {} + } +} diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 6d0db1b79f..15a327cf03 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -11,9 +11,10 @@ use super::models::{ StatusResponse, TenantConfigRequest, TenantCreateRequest, TenantCreateResponse, TenantInfo, TimelineCreateRequest, TimelineInfo, }; +use crate::context::RequestContext; use crate::pgdatadir_mapping::LsnForTimestamp; use crate::tenant::config::TenantConfOpt; -use crate::tenant::{PageReconstructError, Timeline}; +use crate::tenant::{PageReconstructError, Timeline, TimelineRequestContext}; use crate::{config::PageServerConf, tenant::mgr}; use utils::{ auth::JwtAuth, @@ -90,8 +91,9 @@ fn apierror_from_prerror(err: PageReconstructError) -> ApiError { async fn build_timeline_info( timeline: &Arc, include_non_incremental_logical_size: bool, + ctx: &TimelineRequestContext, ) -> anyhow::Result { - let mut info = build_timeline_info_common(timeline)?; + let mut info = build_timeline_info_common(timeline, ctx)?; if include_non_incremental_logical_size { // XXX we should be using spawn_ondemand_logical_size_calculation here. // Otherwise, if someone deletes the timeline / detaches the tenant while @@ -101,6 +103,7 @@ async fn build_timeline_info( .get_current_logical_size_non_incremental( info.last_record_lsn, CancellationToken::new(), + ctx, ) .await?, ); @@ -108,7 +111,10 @@ async fn build_timeline_info( Ok(info) } -fn build_timeline_info_common(timeline: &Arc) -> anyhow::Result { +fn build_timeline_info_common( + timeline: &Arc, + ctx: &TimelineRequestContext, +) -> anyhow::Result { let last_record_lsn = timeline.get_last_record_lsn(); let (wal_source_connstr, last_received_msg_lsn, last_received_msg_ts) = { let guard = timeline.last_received_wal.lock().unwrap(); @@ -128,7 +134,7 @@ fn build_timeline_info_common(timeline: &Arc) -> anyhow::Result None, lsn @ Lsn(_) => Some(lsn), }; - let current_logical_size = match timeline.get_current_logical_size() { + let current_logical_size = match timeline.get_current_logical_size(ctx) { Ok((size, _)) => Some(size), Err(err) => { error!("Timeline info creation failed to get current logical size: {err:?}"); @@ -182,17 +188,20 @@ async fn timeline_create_handler(mut request: Request) -> Result { // Created. Construct a TimelineInfo for it. - let timeline_info = build_timeline_info_common(&new_timeline) + let timeline_ctx = new_timeline.get_context(); + let timeline_info = build_timeline_info_common(&new_timeline, &timeline_ctx) .map_err(ApiError::InternalServerError)?; json_response(StatusCode::CREATED, timeline_info) } @@ -215,13 +224,15 @@ async fn timeline_list_handler(request: Request) -> Result, let mut response_data = Vec::with_capacity(timelines.len()); for timeline in timelines { - let timeline_info = - build_timeline_info(&timeline, include_non_incremental_logical_size) - .await - .context( - "Failed to convert tenant timeline {timeline_id} into the local one: {e:?}", - ) - .map_err(ApiError::InternalServerError)?; + let timeline_ctx = timeline.get_context(); + let timeline_info = build_timeline_info( + &timeline, + include_non_incremental_logical_size, + &timeline_ctx, + ) + .await + .context("Failed to convert tenant timeline {timeline_id} into the local one: {e:?}") + .map_err(ApiError::InternalServerError)?; response_data.push(timeline_info); } @@ -278,11 +289,16 @@ async fn timeline_detail_handler(request: Request) -> Result(timeline_info) } @@ -307,8 +323,9 @@ async fn get_lsn_by_timestamp_handler(request: Request) -> Result) -> Result) -> Result, A let tenant = mgr::get_tenant(tenant_id, true) .await .map_err(ApiError::InternalServerError)?; + let ctx = tenant.get_context(); // this can be long operation, it currently is not backed by any request coalescing or similar let inputs = tenant - .gather_size_inputs() + .gather_size_inputs(&ctx) .await .map_err(ApiError::InternalServerError)?; @@ -505,6 +525,8 @@ fn bad_duration<'a>(field_name: &'static str, value: &'a str) -> impl 'a + Fn() async fn tenant_create_handler(mut request: Request) -> Result, ApiError> { check_permission(&request, None)?; + let ctx = RequestContext::new(); + let request_data: TenantCreateRequest = json_request(&mut request).await?; let mut tenant_conf = TenantConfOpt::default(); @@ -593,7 +615,7 @@ async fn tenant_create_handler(mut request: Request) -> Result { // We created the tenant. Existing API semantics are that the tenant // is Active when this function returns. - if let res @ Err(_) = tenant.wait_to_become_active().await { + if let res @ Err(_) = tenant.wait_to_become_active(&ctx).await { // This shouldn't happen because we just created the tenant directory // in tenant::mgr::create_tenant, and there aren't any remote timelines // to load, so, nothing can really fail during load. @@ -617,6 +639,8 @@ async fn tenant_config_handler(mut request: Request) -> Result) -> Result) -> Result json_response(StatusCode::ACCEPTED, st), Err(st) => json_response(StatusCode::CONFLICT, st), } diff --git a/pageserver/src/import_datadir.rs b/pageserver/src/import_datadir.rs index 3fd4bf12ca..e8b0e0611f 100644 --- a/pageserver/src/import_datadir.rs +++ b/pageserver/src/import_datadir.rs @@ -13,7 +13,7 @@ use tracing::*; use walkdir::WalkDir; use crate::pgdatadir_mapping::*; -use crate::tenant::Timeline; +use crate::tenant::{Timeline, TimelineRequestContext}; use crate::walingest::WalIngest; use crate::walrecord::DecodedWALRecord; use pageserver_api::reltag::{RelTag, SlruKind}; @@ -47,6 +47,7 @@ pub async fn import_timeline_from_postgres_datadir( tline: &Timeline, pgdata_path: &Path, pgdata_lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result<()> { let mut pg_control: Option = None; @@ -69,7 +70,7 @@ pub async fn import_timeline_from_postgres_datadir( let mut file = tokio::fs::File::open(absolute_path).await?; let len = metadata.len() as usize; if let Some(control_file) = - import_file(&mut modification, relative_path, &mut file, len).await? + import_file(&mut modification, relative_path, &mut file, len, ctx).await? { pg_control = Some(control_file); } @@ -99,6 +100,7 @@ pub async fn import_timeline_from_postgres_datadir( tline, Lsn(pg_control.checkPointCopy.redo), pgdata_lsn, + ctx, ) .await?; @@ -113,6 +115,7 @@ async fn import_rel( dboid: Oid, reader: &mut (impl AsyncRead + Send + Sync + Unpin), len: usize, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // Does it look like a relation file? trace!("importing rel file {}", path.display()); @@ -147,7 +150,10 @@ async fn import_rel( // FIXME: use proper error type for this, instead of parsing the error message. // Or better yet, keep track of which relations we've already created // https://github.com/neondatabase/neon/issues/3309 - if let Err(e) = modification.put_rel_creation(rel, nblocks as u32).await { + if let Err(e) = modification + .put_rel_creation(rel, nblocks as u32, ctx) + .await + { if e.to_string().contains("already exists") { debug!("relation {} already exists. we must be extending it", rel); } else { @@ -182,7 +188,7 @@ async fn import_rel( // // If we process rel segments out of order, // put_rel_extend will skip the update. - modification.put_rel_extend(rel, blknum).await?; + modification.put_rel_extend(rel, blknum, ctx).await?; Ok(()) } @@ -195,6 +201,7 @@ async fn import_slru( path: &Path, reader: &mut (impl AsyncRead + Send + Sync + Unpin), len: usize, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { info!("importing slru file {path:?}"); @@ -211,7 +218,7 @@ async fn import_slru( ensure!(nblocks <= pg_constants::SLRU_PAGES_PER_SEGMENT as usize); modification - .put_slru_segment_creation(slru, segno, nblocks as u32) + .put_slru_segment_creation(slru, segno, nblocks as u32, ctx) .await?; let mut rpageno = 0; @@ -252,6 +259,7 @@ async fn import_wal( tline: &Timeline, startpoint: Lsn, endpoint: Lsn, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { use std::io::Read; let mut waldecoder = WalStreamDecoder::new(startpoint, tline.pg_version); @@ -260,7 +268,7 @@ async fn import_wal( let mut offset = startpoint.segment_offset(WAL_SEGMENT_SIZE); let mut last_lsn = startpoint; - let mut walingest = WalIngest::new(tline, startpoint).await?; + let mut walingest = WalIngest::new(tline, startpoint, ctx).await?; while last_lsn <= endpoint { // FIXME: assume postgresql tli 1 for now @@ -297,7 +305,7 @@ async fn import_wal( while last_lsn <= endpoint { if let Some((lsn, recdata)) = waldecoder.poll_decode()? { walingest - .ingest_record(recdata, lsn, &mut modification, &mut decoded) + .ingest_record(recdata, lsn, &mut modification, &mut decoded, ctx) .await?; last_lsn = lsn; @@ -326,6 +334,7 @@ pub async fn import_basebackup_from_tar( tline: &Timeline, reader: &mut (impl AsyncRead + Send + Sync + Unpin), base_lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result<()> { info!("importing base at {base_lsn}"); let mut modification = tline.begin_modification(base_lsn); @@ -344,7 +353,7 @@ pub async fn import_basebackup_from_tar( match header.entry_type() { tokio_tar::EntryType::Regular => { if let Some(res) = - import_file(&mut modification, file_path.as_ref(), &mut entry, len).await? + import_file(&mut modification, file_path.as_ref(), &mut entry, len, ctx).await? { // We found the pg_control file. pg_control = Some(res); @@ -376,13 +385,14 @@ pub async fn import_wal_from_tar( reader: &mut (impl AsyncRead + Send + Sync + Unpin), start_lsn: Lsn, end_lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result<()> { // Set up walingest mutable state let mut waldecoder = WalStreamDecoder::new(start_lsn, tline.pg_version); let mut segno = start_lsn.segment_number(WAL_SEGMENT_SIZE); let mut offset = start_lsn.segment_offset(WAL_SEGMENT_SIZE); let mut last_lsn = start_lsn; - let mut walingest = WalIngest::new(tline, start_lsn).await?; + let mut walingest = WalIngest::new(tline, start_lsn, ctx).await?; // Ingest wal until end_lsn info!("importing wal until {}", end_lsn); @@ -431,7 +441,7 @@ pub async fn import_wal_from_tar( while last_lsn <= end_lsn { if let Some((lsn, recdata)) = waldecoder.poll_decode()? { walingest - .ingest_record(recdata, lsn, &mut modification, &mut decoded) + .ingest_record(recdata, lsn, &mut modification, &mut decoded, ctx) .await?; last_lsn = lsn; @@ -466,6 +476,7 @@ async fn import_file( file_path: &Path, reader: &mut (impl AsyncRead + Send + Sync + Unpin), len: usize, + ctx: &TimelineRequestContext, ) -> Result> { let file_name = match file_path.file_name() { Some(name) => name.to_string_lossy(), @@ -498,14 +509,16 @@ async fn import_file( } "pg_filenode.map" => { let bytes = read_all_bytes(reader).await?; - modification.put_relmap_file(spcnode, dbnode, bytes).await?; + modification + .put_relmap_file(spcnode, dbnode, bytes, ctx) + .await?; debug!("imported relmap file") } "PG_VERSION" => { debug!("ignored PG_VERSION file"); } _ => { - import_rel(modification, file_path, spcnode, dbnode, reader, len).await?; + import_rel(modification, file_path, spcnode, dbnode, reader, len, ctx).await?; debug!("imported rel creation"); } } @@ -521,38 +534,40 @@ async fn import_file( match file_name.as_ref() { "pg_filenode.map" => { let bytes = read_all_bytes(reader).await?; - modification.put_relmap_file(spcnode, dbnode, bytes).await?; + modification + .put_relmap_file(spcnode, dbnode, bytes, ctx) + .await?; debug!("imported relmap file") } "PG_VERSION" => { debug!("ignored PG_VERSION file"); } _ => { - import_rel(modification, file_path, spcnode, dbnode, reader, len).await?; + import_rel(modification, file_path, spcnode, dbnode, reader, len, ctx).await?; debug!("imported rel creation"); } } } else if file_path.starts_with("pg_xact") { let slru = SlruKind::Clog; - import_slru(modification, slru, file_path, reader, len).await?; + import_slru(modification, slru, file_path, reader, len, ctx).await?; debug!("imported clog slru"); } else if file_path.starts_with("pg_multixact/offsets") { let slru = SlruKind::MultiXactOffsets; - import_slru(modification, slru, file_path, reader, len).await?; + import_slru(modification, slru, file_path, reader, len, ctx).await?; debug!("imported multixact offsets slru"); } else if file_path.starts_with("pg_multixact/members") { let slru = SlruKind::MultiXactMembers; - import_slru(modification, slru, file_path, reader, len).await?; + import_slru(modification, slru, file_path, reader, len, ctx).await?; debug!("imported multixact members slru"); } else if file_path.starts_with("pg_twophase") { let xid = u32::from_str_radix(file_name.as_ref(), 16)?; let bytes = read_all_bytes(reader).await?; modification - .put_twophase_file(xid, Bytes::copy_from_slice(&bytes[..])) + .put_twophase_file(xid, Bytes::copy_from_slice(&bytes[..]), ctx) .await?; debug!("imported twophase file"); } else if file_path.starts_with("pg_wal") { diff --git a/pageserver/src/lib.rs b/pageserver/src/lib.rs index 91cde477ad..3ea2165da9 100644 --- a/pageserver/src/lib.rs +++ b/pageserver/src/lib.rs @@ -2,6 +2,7 @@ mod auth; pub mod basebackup; pub mod config; pub mod consumption_metrics; +pub mod context; pub mod http; pub mod import_datadir; pub mod keyspace; diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index 344a8d1c00..32ef056220 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -43,18 +43,22 @@ use utils::{ use crate::auth::check_permission; use crate::basebackup; use crate::config::PageServerConf; +use crate::context::RequestContext; use crate::import_datadir::import_wal_from_tar; use crate::metrics::{LIVE_CONNECTIONS_COUNT, SMGR_QUERY_TIME}; use crate::task_mgr; use crate::task_mgr::TaskKind; use crate::tenant::mgr; -use crate::tenant::{Tenant, Timeline}; +use crate::tenant::{Tenant, Timeline, TimelineRequestContext}; use crate::trace::Tracer; use postgres_ffi::pg_constants::DEFAULTTABLESPACE_OID; use postgres_ffi::BLCKSZ; -fn copyin_stream(pgb: &mut PostgresBackend) -> impl Stream> + '_ { +fn copyin_stream<'a>( + pgb: &'a mut PostgresBackend, + ctx: &'a RequestContext, +) -> impl Stream> + 'a { async_stream::try_stream! { loop { let msg = tokio::select! { @@ -146,6 +150,8 @@ pub async fn libpq_listener_main( debug!("accepted connection from {}", peer_addr); let local_auth = auth.clone(); + let connection_ctx = RequestContext::new(); + // PageRequestHandler tasks are not associated with any particular // timeline in the task manager. In practice most connections will // only deal with a particular timeline, but we don't know which one @@ -157,7 +163,7 @@ pub async fn libpq_listener_main( None, "serving compute connection task", false, - page_service_conn_main(conf, local_auth, socket, auth_type), + page_service_conn_main(conf, local_auth, socket, auth_type, connection_ctx), ); } Err(err) => { @@ -177,6 +183,7 @@ async fn page_service_conn_main( auth: Option>, socket: tokio::net::TcpStream, auth_type: AuthType, + connection_ctx: RequestContext, ) -> anyhow::Result<()> { // Immediately increment the gauge, then create a job to decrement it on task exit. // One of the pros of `defer!` is that this will *most probably* @@ -191,7 +198,7 @@ async fn page_service_conn_main( .set_nodelay(true) .context("could not set TCP_NODELAY")?; - let mut conn_handler = PageServerHandler::new(conf, auth); + let mut conn_handler = PageServerHandler::new(conf, auth, connection_ctx); let pgbackend = PostgresBackend::new(socket, auth_type, None)?; let result = pgbackend @@ -255,20 +262,27 @@ struct PageServerHandler { _conf: &'static PageServerConf, auth: Option>, claims: Option, + + connection_ctx: RequestContext, } impl PageServerHandler { - pub fn new(conf: &'static PageServerConf, auth: Option>) -> Self { + pub fn new( + conf: &'static PageServerConf, + auth: Option>, + connection_ctx: RequestContext, + ) -> Self { PageServerHandler { _conf: conf, auth, claims: None, + connection_ctx, } } #[instrument(skip(self, pgb))] async fn handle_pagerequests( - &self, + &mut self, pgb: &mut PostgresBackend, tenant_id: TenantId, timeline_id: TimelineId, @@ -278,7 +292,7 @@ impl PageServerHandler { task_mgr::associate_with(Some(tenant_id), Some(timeline_id)); // Make request tracer if needed - let tenant = get_active_tenant_with_timeout(tenant_id).await?; + let tenant = get_active_tenant_with_timeout(tenant_id, &self.connection_ctx).await?; let mut tracer = if tenant.get_trace_read_requests() { let connection_id = ConnectionId::generate(); let path = tenant @@ -291,6 +305,7 @@ impl PageServerHandler { // Check that the timeline exists let timeline = tenant.get_timeline(timeline_id, true)?; + let ctx = timeline.get_context(); // switch client to COPYBOTH pgb.write_message(&BeMessage::CopyBothResponse)?; @@ -332,19 +347,21 @@ impl PageServerHandler { let response = match neon_fe_msg { PagestreamFeMessage::Exists(req) => { let _timer = metrics.get_rel_exists.start_timer(); - self.handle_get_rel_exists_request(&timeline, &req).await + self.handle_get_rel_exists_request(&timeline, &req, &ctx) + .await } PagestreamFeMessage::Nblocks(req) => { let _timer = metrics.get_rel_size.start_timer(); - self.handle_get_nblocks_request(&timeline, &req).await + self.handle_get_nblocks_request(&timeline, &req, &ctx).await } PagestreamFeMessage::GetPage(req) => { let _timer = metrics.get_page_at_lsn.start_timer(); - self.handle_get_page_at_lsn_request(&timeline, &req).await + self.handle_get_page_at_lsn_request(&timeline, &req, &ctx) + .await } PagestreamFeMessage::DbSize(req) => { let _timer = metrics.get_db_size.start_timer(); - self.handle_db_size_request(&timeline, &req).await + self.handle_db_size_request(&timeline, &req, &ctx).await } }; @@ -376,8 +393,10 @@ impl PageServerHandler { task_mgr::associate_with(Some(tenant_id), Some(timeline_id)); // Create empty timeline info!("creating new timeline"); - let tenant = get_active_tenant_with_timeout(tenant_id).await?; - let timeline = tenant.create_empty_timeline(timeline_id, base_lsn, pg_version)?; + let tenant = get_active_tenant_with_timeout(tenant_id, &self.connection_ctx).await?; + let tenant_ctx = tenant.get_context(); + let (timeline, ctx) = + tenant.create_empty_timeline(timeline_id, base_lsn, pg_version, &tenant_ctx)?; // TODO mark timeline as not ready until it reaches end_lsn. // We might have some wal to import as well, and we should prevent compute @@ -394,9 +413,9 @@ impl PageServerHandler { pgb.write_message(&BeMessage::CopyInResponse)?; pgb.flush().await?; - let mut copyin_stream = Box::pin(copyin_stream(pgb)); + let mut copyin_stream = Box::pin(copyin_stream(pgb, &ctx)); timeline - .import_basebackup_from_tar(&mut copyin_stream, base_lsn) + .import_basebackup_from_tar(&mut copyin_stream, base_lsn, &ctx) .await?; // Drain the rest of the Copy data @@ -429,7 +448,9 @@ impl PageServerHandler { ) -> Result<(), QueryError> { task_mgr::associate_with(Some(tenant_id), Some(timeline_id)); - let timeline = get_active_timeline_with_timeout(tenant_id, timeline_id).await?; + let timeline = + get_active_timeline_with_timeout(tenant_id, timeline_id, &self.connection_ctx).await?; + let ctx = timeline.get_context(); let last_record_lsn = timeline.get_last_record_lsn(); if last_record_lsn != start_lsn { return Err(QueryError::Other( @@ -444,9 +465,9 @@ impl PageServerHandler { info!("importing wal"); pgb.write_message(&BeMessage::CopyInResponse)?; pgb.flush().await?; - let mut copyin_stream = Box::pin(copyin_stream(pgb)); + let mut copyin_stream = Box::pin(copyin_stream(pgb, &ctx)); let mut reader = tokio_util::io::StreamReader::new(&mut copyin_stream); - import_wal_from_tar(&timeline, &mut reader, start_lsn, end_lsn).await?; + import_wal_from_tar(&timeline, &mut reader, start_lsn, end_lsn, &ctx).await?; info!("wal import complete"); // Drain the rest of the Copy data @@ -492,6 +513,7 @@ impl PageServerHandler { mut lsn: Lsn, latest: bool, latest_gc_cutoff_lsn: &RcuReadGuard, + ctx: &TimelineRequestContext, ) -> anyhow::Result { if latest { // Latest page version was requested. If LSN is given, it is a hint @@ -515,7 +537,7 @@ impl PageServerHandler { if lsn <= last_record_lsn { lsn = last_record_lsn; } else { - timeline.wait_lsn(lsn).await?; + timeline.wait_lsn(lsn, ctx).await?; // Since we waited for 'lsn' to arrive, that is now the last // record LSN. (Or close enough for our purposes; the // last-record LSN can advance immediately after we return @@ -525,7 +547,7 @@ impl PageServerHandler { if lsn == Lsn(0) { anyhow::bail!("invalid LSN(0) in request"); } - timeline.wait_lsn(lsn).await?; + timeline.wait_lsn(lsn, ctx).await?; } anyhow::ensure!( lsn >= **latest_gc_cutoff_lsn, @@ -535,52 +557,60 @@ impl PageServerHandler { Ok(lsn) } - #[instrument(skip(self, timeline, req), fields(rel = %req.rel, req_lsn = %req.lsn))] + #[instrument(skip(self, timeline, req, ctx), fields(rel = %req.rel, req_lsn = %req.lsn))] async fn handle_get_rel_exists_request( &self, timeline: &Timeline, req: &PagestreamExistsRequest, + ctx: &TimelineRequestContext, ) -> anyhow::Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); - let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn) - .await?; + let lsn = + Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) + .await?; - let exists = timeline.get_rel_exists(req.rel, lsn, req.latest).await?; + let exists = timeline + .get_rel_exists(req.rel, lsn, req.latest, ctx) + .await?; Ok(PagestreamBeMessage::Exists(PagestreamExistsResponse { exists, })) } - #[instrument(skip(self, timeline, req), fields(rel = %req.rel, req_lsn = %req.lsn))] + #[instrument(skip(self, timeline, req, ctx), fields(rel = %req.rel, req_lsn = %req.lsn))] async fn handle_get_nblocks_request( &self, timeline: &Timeline, req: &PagestreamNblocksRequest, + ctx: &TimelineRequestContext, ) -> anyhow::Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); - let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn) - .await?; + let lsn = + Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) + .await?; - let n_blocks = timeline.get_rel_size(req.rel, lsn, req.latest).await?; + let n_blocks = timeline.get_rel_size(req.rel, lsn, req.latest, ctx).await?; Ok(PagestreamBeMessage::Nblocks(PagestreamNblocksResponse { n_blocks, })) } - #[instrument(skip(self, timeline, req), fields(dbnode = %req.dbnode, req_lsn = %req.lsn))] + #[instrument(skip(self, timeline, req, ctx), fields(dbnode = %req.dbnode, req_lsn = %req.lsn))] async fn handle_db_size_request( &self, timeline: &Timeline, req: &PagestreamDbSizeRequest, + ctx: &TimelineRequestContext, ) -> anyhow::Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); - let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn) - .await?; + let lsn = + Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) + .await?; let total_blocks = timeline - .get_db_size(DEFAULTTABLESPACE_OID, req.dbnode, lsn, req.latest) + .get_db_size(DEFAULTTABLESPACE_OID, req.dbnode, lsn, req.latest, ctx) .await?; let db_size = total_blocks as i64 * BLCKSZ as i64; @@ -589,15 +619,17 @@ impl PageServerHandler { })) } - #[instrument(skip(self, timeline, req), fields(rel = %req.rel, blkno = %req.blkno, req_lsn = %req.lsn))] + #[instrument(skip(self, timeline, req, ctx), fields(rel = %req.rel, blkno = %req.blkno, req_lsn = %req.lsn))] async fn handle_get_page_at_lsn_request( &self, timeline: &Timeline, req: &PagestreamGetPageRequest, + ctx: &TimelineRequestContext, ) -> anyhow::Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); - let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn) - .await?; + let lsn = + Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) + .await?; /* // Add a 1s delay to some requests. The delay helps the requests to // hit the race condition from github issue #1047 more easily. @@ -608,7 +640,7 @@ impl PageServerHandler { */ let page = timeline - .get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest) + .get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest, ctx) .await?; Ok(PagestreamBeMessage::GetPage(PagestreamGetPageResponse { @@ -618,7 +650,7 @@ impl PageServerHandler { #[instrument(skip(self, pgb))] async fn handle_basebackup_request( - &self, + &mut self, pgb: &mut PostgresBackend, tenant_id: TenantId, timeline_id: TimelineId, @@ -627,12 +659,15 @@ impl PageServerHandler { full_backup: bool, ) -> anyhow::Result<()> { // check that the timeline exists - let timeline = get_active_timeline_with_timeout(tenant_id, timeline_id).await?; + let timeline = + get_active_timeline_with_timeout(tenant_id, timeline_id, &self.connection_ctx).await?; + let ctx = timeline.get_context(); + let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); if let Some(lsn) = lsn { // Backup was requested at a particular LSN. Wait for it to arrive. info!("waiting for {}", lsn); - timeline.wait_lsn(lsn).await?; + timeline.wait_lsn(lsn, &ctx).await?; timeline .check_lsn_is_in_scope(lsn, &latest_gc_cutoff_lsn) .context("invalid basebackup lsn")?; @@ -645,8 +680,15 @@ impl PageServerHandler { // Send a tarball of the latest layer on the timeline { let mut writer = pgb.copyout_writer(); - basebackup::send_basebackup_tarball(&mut writer, &timeline, lsn, prev_lsn, full_backup) - .await?; + basebackup::send_basebackup_tarball( + &mut writer, + &timeline, + lsn, + prev_lsn, + full_backup, + &ctx, + ) + .await?; } pgb.write_message(&BeMessage::CopyDone)?; @@ -784,7 +826,9 @@ impl postgres_backend_async::Handler for PageServerHandler { .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?; self.check_permission(Some(tenant_id))?; - let timeline = get_active_timeline_with_timeout(tenant_id, timeline_id).await?; + let timeline = + get_active_timeline_with_timeout(tenant_id, timeline_id, &self.connection_ctx) + .await?; let end_of_timeline = timeline.get_last_record_rlsn(); @@ -944,7 +988,7 @@ impl postgres_backend_async::Handler for PageServerHandler { self.check_permission(Some(tenant_id))?; - let tenant = get_active_tenant_with_timeout(tenant_id).await?; + let tenant = get_active_tenant_with_timeout(tenant_id, &self.connection_ctx).await?; pgb.write_message(&BeMessage::RowDescription(&[ RowDescriptor::int8_col(b"checkpoint_distance"), RowDescriptor::int8_col(b"checkpoint_timeout"), @@ -995,9 +1039,17 @@ impl postgres_backend_async::Handler for PageServerHandler { /// If the tenant is Loading, waits for it to become Active, for up to 30 s. That /// ensures that queries don't fail immediately after pageserver startup, because /// all tenants are still loading. -async fn get_active_tenant_with_timeout(tenant_id: TenantId) -> anyhow::Result> { +async fn get_active_tenant_with_timeout( + tenant_id: TenantId, + parent_ctx: &RequestContext, +) -> anyhow::Result> { let tenant = mgr::get_tenant(tenant_id, false).await?; - match tokio::time::timeout(Duration::from_secs(30), tenant.wait_to_become_active()).await { + match tokio::time::timeout( + Duration::from_secs(30), + tenant.wait_to_become_active(parent_ctx), + ) + .await + { Ok(wait_result) => wait_result // no .context(), the error message is good enough and some tests depend on it .map(move |()| tenant), @@ -1009,8 +1061,9 @@ async fn get_active_tenant_with_timeout(tenant_id: TenantId) -> anyhow::Result anyhow::Result> { - get_active_tenant_with_timeout(tenant_id) + get_active_tenant_with_timeout(tenant_id, ctx) .await .and_then(|tenant| tenant.get_timeline(timeline_id, true)) } diff --git a/pageserver/src/pgdatadir_mapping.rs b/pageserver/src/pgdatadir_mapping.rs index 6ae70e3a30..9a15b40e16 100644 --- a/pageserver/src/pgdatadir_mapping.rs +++ b/pageserver/src/pgdatadir_mapping.rs @@ -6,9 +6,9 @@ //! walingest.rs handles a few things like implicit relation creation and extension. //! Clarify that) //! -use super::tenant::{PageReconstructError, Timeline}; use crate::keyspace::{KeySpace, KeySpaceAccum}; use crate::repository::*; +use crate::tenant::{PageReconstructError, Timeline, TimelineRequestContext}; use crate::walrecord::NeonWalRecord; use anyhow::Context; use bytes::{Buf, Bytes}; @@ -97,6 +97,7 @@ impl Timeline { blknum: BlockNumber, lsn: Lsn, latest: bool, + ctx: &TimelineRequestContext, ) -> Result { if tag.relnode == 0 { return Err(PageReconstructError::Other(anyhow::anyhow!( @@ -104,7 +105,7 @@ impl Timeline { ))); } - let nblocks = self.get_rel_size(tag, lsn, latest).await?; + let nblocks = self.get_rel_size(tag, lsn, latest, ctx).await?; if blknum >= nblocks { debug!( "read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page", @@ -114,7 +115,7 @@ impl Timeline { } let key = rel_block_to_key(tag, blknum); - self.get(key, lsn).await + self.get(key, lsn, ctx).await } // Get size of a database in blocks @@ -124,13 +125,14 @@ impl Timeline { dbnode: Oid, lsn: Lsn, latest: bool, + ctx: &TimelineRequestContext, ) -> Result { let mut total_blocks = 0; - let rels = self.list_rels(spcnode, dbnode, lsn).await?; + let rels = self.list_rels(spcnode, dbnode, lsn, ctx).await?; for rel in rels { - let n_blocks = self.get_rel_size(rel, lsn, latest).await?; + let n_blocks = self.get_rel_size(rel, lsn, latest, ctx).await?; total_blocks += n_blocks as usize; } Ok(total_blocks) @@ -142,6 +144,7 @@ impl Timeline { tag: RelTag, lsn: Lsn, latest: bool, + ctx: &TimelineRequestContext, ) -> Result { if tag.relnode == 0 { return Err(PageReconstructError::Other(anyhow::anyhow!( @@ -154,7 +157,7 @@ impl Timeline { } if (tag.forknum == FSM_FORKNUM || tag.forknum == VISIBILITYMAP_FORKNUM) - && !self.get_rel_exists(tag, lsn, latest).await? + && !self.get_rel_exists(tag, lsn, latest, ctx).await? { // FIXME: Postgres sometimes calls smgrcreate() to create // FSM, and smgrnblocks() on it immediately afterwards, @@ -164,7 +167,7 @@ impl Timeline { } let key = rel_size_to_key(tag); - let mut buf = self.get(key, lsn).await?; + let mut buf = self.get(key, lsn, ctx).await?; let nblocks = buf.get_u32_le(); if latest { @@ -186,6 +189,7 @@ impl Timeline { tag: RelTag, lsn: Lsn, _latest: bool, + ctx: &TimelineRequestContext, ) -> Result { if tag.relnode == 0 { return Err(PageReconstructError::Other(anyhow::anyhow!( @@ -199,7 +203,7 @@ impl Timeline { } // fetch directory listing let key = rel_dir_to_key(tag.spcnode, tag.dbnode); - let buf = self.get(key, lsn).await?; + let buf = self.get(key, lsn, ctx).await?; match RelDirectory::des(&buf).context("deserialization failure") { Ok(dir) => { @@ -216,10 +220,11 @@ impl Timeline { spcnode: Oid, dbnode: Oid, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result, PageReconstructError> { // fetch directory listing let key = rel_dir_to_key(spcnode, dbnode); - let buf = self.get(key, lsn).await?; + let buf = self.get(key, lsn, ctx).await?; match RelDirectory::des(&buf).context("deserialization failure") { Ok(dir) => { @@ -244,9 +249,10 @@ impl Timeline { segno: u32, blknum: BlockNumber, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result { let key = slru_block_to_key(kind, segno, blknum); - self.get(key, lsn).await + self.get(key, lsn, ctx).await } /// Get size of an SLRU segment @@ -255,9 +261,10 @@ impl Timeline { kind: SlruKind, segno: u32, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result { let key = slru_segment_size_to_key(kind, segno); - let mut buf = self.get(key, lsn).await?; + let mut buf = self.get(key, lsn, ctx).await?; Ok(buf.get_u32_le()) } @@ -267,10 +274,11 @@ impl Timeline { kind: SlruKind, segno: u32, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result { // fetch directory listing let key = slru_dir_to_key(kind); - let buf = self.get(key, lsn).await?; + let buf = self.get(key, lsn, ctx).await?; match SlruSegmentDirectory::des(&buf).context("deserialization failure") { Ok(dir) => { @@ -291,6 +299,7 @@ impl Timeline { pub async fn find_lsn_for_timestamp( &self, search_timestamp: TimestampTz, + ctx: &TimelineRequestContext, ) -> Result { let gc_cutoff_lsn_guard = self.get_latest_gc_cutoff_lsn(); let min_lsn = *gc_cutoff_lsn_guard; @@ -313,6 +322,7 @@ impl Timeline { Lsn(mid * 8), &mut found_smaller, &mut found_larger, + ctx, ) .await?; @@ -362,14 +372,18 @@ impl Timeline { probe_lsn: Lsn, found_smaller: &mut bool, found_larger: &mut bool, + ctx: &TimelineRequestContext, ) -> Result { - for segno in self.list_slru_segments(SlruKind::Clog, probe_lsn).await? { + for segno in self + .list_slru_segments(SlruKind::Clog, probe_lsn, ctx) + .await? + { let nblocks = self - .get_slru_segment_size(SlruKind::Clog, segno, probe_lsn) + .get_slru_segment_size(SlruKind::Clog, segno, probe_lsn, ctx) .await?; for blknum in (0..nblocks).rev() { let clog_page = self - .get_slru_page_at_lsn(SlruKind::Clog, segno, blknum, probe_lsn) + .get_slru_page_at_lsn(SlruKind::Clog, segno, blknum, probe_lsn, ctx) .await?; if clog_page.len() == BLCKSZ as usize + 8 { @@ -394,11 +408,12 @@ impl Timeline { &self, kind: SlruKind, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result, PageReconstructError> { // fetch directory entry let key = slru_dir_to_key(kind); - let buf = self.get(key, lsn).await?; + let buf = self.get(key, lsn, ctx).await?; match SlruSegmentDirectory::des(&buf).context("deserialization failure") { Ok(dir) => Ok(dir.segments), Err(e) => Err(PageReconstructError::from(e)), @@ -410,18 +425,20 @@ impl Timeline { spcnode: Oid, dbnode: Oid, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result { let key = relmap_file_key(spcnode, dbnode); - self.get(key, lsn).await + self.get(key, lsn, ctx).await } pub async fn list_dbdirs( &self, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result, PageReconstructError> { // fetch directory entry - let buf = self.get(DBDIR_KEY, lsn).await?; + let buf = self.get(DBDIR_KEY, lsn, ctx).await?; match DbDirectory::des(&buf).context("deserialization failure") { Ok(dir) => Ok(dir.dbdirs), @@ -433,18 +450,20 @@ impl Timeline { &self, xid: TransactionId, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result { let key = twophase_file_key(xid); - let buf = self.get(key, lsn).await?; + let buf = self.get(key, lsn, ctx).await?; Ok(buf) } pub async fn list_twophase_files( &self, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> Result, PageReconstructError> { // fetch directory entry - let buf = self.get(TWOPHASEDIR_KEY, lsn).await?; + let buf = self.get(TWOPHASEDIR_KEY, lsn, ctx).await?; match TwoPhaseDirectory::des(&buf).context("deserialization failure") { Ok(dir) => Ok(dir.xids), @@ -452,12 +471,20 @@ impl Timeline { } } - pub async fn get_control_file(&self, lsn: Lsn) -> Result { - self.get(CONTROLFILE_KEY, lsn).await + pub async fn get_control_file( + &self, + lsn: Lsn, + ctx: &TimelineRequestContext, + ) -> Result { + self.get(CONTROLFILE_KEY, lsn, ctx).await } - pub async fn get_checkpoint(&self, lsn: Lsn) -> Result { - self.get(CHECKPOINT_KEY, lsn).await + pub async fn get_checkpoint( + &self, + lsn: Lsn, + ctx: &TimelineRequestContext, + ) -> Result { + self.get(CHECKPOINT_KEY, lsn, ctx).await } /// Does the same as get_current_logical_size but counted on demand. @@ -469,15 +496,16 @@ impl Timeline { &self, lsn: Lsn, cancel: CancellationToken, + ctx: &TimelineRequestContext, ) -> Result { // Fetch list of database dirs and iterate them - let buf = self.get(DBDIR_KEY, lsn).await.context("read dbdir")?; + let buf = self.get(DBDIR_KEY, lsn, ctx).await.context("read dbdir")?; let dbdir = DbDirectory::des(&buf).context("deserialize db directory")?; let mut total_size: u64 = 0; for (spcnode, dbnode) in dbdir.dbdirs.keys() { for rel in self - .list_rels(*spcnode, *dbnode, lsn) + .list_rels(*spcnode, *dbnode, lsn, ctx) .await .context("list rels")? { @@ -486,7 +514,7 @@ impl Timeline { } let relsize_key = rel_size_to_key(rel); let mut buf = self - .get(relsize_key, lsn) + .get(relsize_key, lsn, ctx) .await .context("read relation size of {rel:?}")?; let relsize = buf.get_u32_le(); @@ -501,7 +529,11 @@ impl Timeline { /// Get a KeySpace that covers all the Keys that are in use at the given LSN. /// Anything that's not listed maybe removed from the underlying storage (from /// that LSN forwards). - pub async fn collect_keyspace(&self, lsn: Lsn) -> anyhow::Result { + pub async fn collect_keyspace( + &self, + lsn: Lsn, + ctx: &TimelineRequestContext, + ) -> anyhow::Result { // Iterate through key ranges, greedily packing them into partitions let mut result = KeySpaceAccum::new(); @@ -509,7 +541,7 @@ impl Timeline { result.add_key(DBDIR_KEY); // Fetch list of database dirs and iterate them - let buf = self.get(DBDIR_KEY, lsn).await?; + let buf = self.get(DBDIR_KEY, lsn, ctx).await?; let dbdir = DbDirectory::des(&buf).context("deserialization failure")?; let mut dbs: Vec<(Oid, Oid)> = dbdir.dbdirs.keys().cloned().collect(); @@ -519,14 +551,14 @@ impl Timeline { result.add_key(rel_dir_to_key(spcnode, dbnode)); let mut rels: Vec = self - .list_rels(spcnode, dbnode, lsn) + .list_rels(spcnode, dbnode, lsn, ctx) .await? .into_iter() .collect(); rels.sort_unstable(); for rel in rels { let relsize_key = rel_size_to_key(rel); - let mut buf = self.get(relsize_key, lsn).await?; + let mut buf = self.get(relsize_key, lsn, ctx).await?; let relsize = buf.get_u32_le(); result.add_range(rel_block_to_key(rel, 0)..rel_block_to_key(rel, relsize)); @@ -542,13 +574,13 @@ impl Timeline { ] { let slrudir_key = slru_dir_to_key(kind); result.add_key(slrudir_key); - let buf = self.get(slrudir_key, lsn).await?; + let buf = self.get(slrudir_key, lsn, ctx).await?; let dir = SlruSegmentDirectory::des(&buf).context("deserialization failure")?; let mut segments: Vec = dir.segments.iter().cloned().collect(); segments.sort_unstable(); for segno in segments { let segsize_key = slru_segment_size_to_key(kind, segno); - let mut buf = self.get(segsize_key, lsn).await?; + let mut buf = self.get(segsize_key, lsn, ctx).await?; let segsize = buf.get_u32_le(); result.add_range( @@ -560,7 +592,7 @@ impl Timeline { // Then pg_twophase result.add_key(TWOPHASEDIR_KEY); - let buf = self.get(TWOPHASEDIR_KEY, lsn).await?; + let buf = self.get(TWOPHASEDIR_KEY, lsn, ctx).await?; let twophase_dir = TwoPhaseDirectory::des(&buf).context("deserialization failure")?; let mut xids: Vec = twophase_dir.xids.iter().cloned().collect(); xids.sort_unstable(); @@ -723,9 +755,10 @@ impl<'a> DatadirModification<'a> { spcnode: Oid, dbnode: Oid, img: Bytes, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // Add it to the directory (if it doesn't exist already) - let buf = self.get(DBDIR_KEY).await?; + let buf = self.get(DBDIR_KEY, ctx).await?; let mut dbdir = DbDirectory::des(&buf)?; let r = dbdir.dbdirs.insert((spcnode, dbnode), true); @@ -755,9 +788,10 @@ impl<'a> DatadirModification<'a> { &mut self, xid: TransactionId, img: Bytes, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // Add it to the directory entry - let buf = self.get(TWOPHASEDIR_KEY).await?; + let buf = self.get(TWOPHASEDIR_KEY, ctx).await?; let mut dir = TwoPhaseDirectory::des(&buf)?; if !dir.xids.insert(xid) { anyhow::bail!("twophase file for xid {} already exists", xid); @@ -781,16 +815,21 @@ impl<'a> DatadirModification<'a> { Ok(()) } - pub async fn drop_dbdir(&mut self, spcnode: Oid, dbnode: Oid) -> anyhow::Result<()> { + pub async fn drop_dbdir( + &mut self, + spcnode: Oid, + dbnode: Oid, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<()> { let req_lsn = self.tline.get_last_record_lsn(); let total_blocks = self .tline - .get_db_size(spcnode, dbnode, req_lsn, true) + .get_db_size(spcnode, dbnode, req_lsn, true, ctx) .await?; // Remove entry from dbdir - let buf = self.get(DBDIR_KEY).await?; + let buf = self.get(DBDIR_KEY, ctx).await?; let mut dir = DbDirectory::des(&buf)?; if dir.dbdirs.remove(&(spcnode, dbnode)).is_some() { let buf = DbDirectory::ser(&dir)?; @@ -817,11 +856,12 @@ impl<'a> DatadirModification<'a> { &mut self, rel: RelTag, nblocks: BlockNumber, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { anyhow::ensure!(rel.relnode != 0, "invalid relnode"); // It's possible that this is the first rel for this db in this // tablespace. Create the reldir entry for it if so. - let mut dbdir = DbDirectory::des(&self.get(DBDIR_KEY).await?)?; + let mut dbdir = DbDirectory::des(&self.get(DBDIR_KEY, ctx).await?)?; let rel_dir_key = rel_dir_to_key(rel.spcnode, rel.dbnode); let mut rel_dir = if dbdir.dbdirs.get(&(rel.spcnode, rel.dbnode)).is_none() { // Didn't exist. Update dbdir @@ -833,7 +873,7 @@ impl<'a> DatadirModification<'a> { RelDirectory::default() } else { // reldir already exists, fetch it - RelDirectory::des(&self.get(rel_dir_key).await?)? + RelDirectory::des(&self.get(rel_dir_key, ctx).await?)? }; // Add the new relation to the rel directory entry, and write it back @@ -865,13 +905,14 @@ impl<'a> DatadirModification<'a> { &mut self, rel: RelTag, nblocks: BlockNumber, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { anyhow::ensure!(rel.relnode != 0, "invalid relnode"); let last_lsn = self.tline.get_last_record_lsn(); - if self.tline.get_rel_exists(rel, last_lsn, true).await? { + if self.tline.get_rel_exists(rel, last_lsn, true, ctx).await? { let size_key = rel_size_to_key(rel); // Fetch the old size first - let old_size = self.get(size_key).await?.get_u32_le(); + let old_size = self.get(size_key, ctx).await?.get_u32_le(); // Update the entry with the new size. let buf = nblocks.to_le_bytes(); @@ -895,12 +936,13 @@ impl<'a> DatadirModification<'a> { &mut self, rel: RelTag, nblocks: BlockNumber, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { anyhow::ensure!(rel.relnode != 0, "invalid relnode"); // Put size let size_key = rel_size_to_key(rel); - let old_size = self.get(size_key).await?.get_u32_le(); + let old_size = self.get(size_key, ctx).await?.get_u32_le(); // only extend relation here. never decrease the size if nblocks > old_size { @@ -916,12 +958,16 @@ impl<'a> DatadirModification<'a> { } /// Drop a relation. - pub async fn put_rel_drop(&mut self, rel: RelTag) -> anyhow::Result<()> { + pub async fn put_rel_drop( + &mut self, + rel: RelTag, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<()> { anyhow::ensure!(rel.relnode != 0, "invalid relnode"); // Remove it from the directory entry let dir_key = rel_dir_to_key(rel.spcnode, rel.dbnode); - let buf = self.get(dir_key).await?; + let buf = self.get(dir_key, ctx).await?; let mut dir = RelDirectory::des(&buf)?; if dir.rels.remove(&(rel.relnode, rel.forknum)) { @@ -932,7 +978,7 @@ impl<'a> DatadirModification<'a> { // update logical size let size_key = rel_size_to_key(rel); - let old_size = self.get(size_key).await?.get_u32_le(); + let old_size = self.get(size_key, ctx).await?.get_u32_le(); self.pending_nblocks -= old_size as i64; // Remove enty from relation size cache @@ -949,10 +995,11 @@ impl<'a> DatadirModification<'a> { kind: SlruKind, segno: u32, nblocks: BlockNumber, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // Add it to the directory entry let dir_key = slru_dir_to_key(kind); - let buf = self.get(dir_key).await?; + let buf = self.get(dir_key, ctx).await?; let mut dir = SlruSegmentDirectory::des(&buf)?; if !dir.segments.insert(segno) { @@ -988,10 +1035,15 @@ impl<'a> DatadirModification<'a> { } /// This method is used for marking truncated SLRU files - pub async fn drop_slru_segment(&mut self, kind: SlruKind, segno: u32) -> anyhow::Result<()> { + pub async fn drop_slru_segment( + &mut self, + kind: SlruKind, + segno: u32, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<()> { // Remove it from the directory entry let dir_key = slru_dir_to_key(kind); - let buf = self.get(dir_key).await?; + let buf = self.get(dir_key, ctx).await?; let mut dir = SlruSegmentDirectory::des(&buf)?; if !dir.segments.remove(&segno) { @@ -1015,9 +1067,13 @@ impl<'a> DatadirModification<'a> { } /// This method is used for marking truncated SLRU files - pub async fn drop_twophase_file(&mut self, xid: TransactionId) -> anyhow::Result<()> { + pub async fn drop_twophase_file( + &mut self, + xid: TransactionId, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<()> { // Remove it from the directory entry - let buf = self.get(TWOPHASEDIR_KEY).await?; + let buf = self.get(TWOPHASEDIR_KEY, ctx).await?; let mut dir = TwoPhaseDirectory::des(&buf)?; if !dir.xids.remove(&xid) { @@ -1111,7 +1167,11 @@ impl<'a> DatadirModification<'a> { // Internal helper functions to batch the modifications - async fn get(&self, key: Key) -> Result { + async fn get( + &self, + key: Key, + ctx: &TimelineRequestContext, + ) -> Result { // Have we already updated the same key? Read the pending updated // version in that case. // @@ -1132,7 +1192,7 @@ impl<'a> DatadirModification<'a> { } } else { let lsn = Lsn::max(self.tline.get_last_record_lsn(), self.lsn); - self.tline.get(key, lsn).await + self.tline.get(key, lsn, ctx).await } } @@ -1539,17 +1599,18 @@ fn is_slru_block_key(key: Key) -> bool { #[cfg(test)] pub fn create_test_timeline( - tenant: &crate::tenant::Tenant, + tenant: &std::sync::Arc, timeline_id: utils::id::TimelineId, pg_version: u32, -) -> anyhow::Result> { - let tline = tenant - .create_empty_timeline(timeline_id, Lsn(8), pg_version)? - .initialize()?; + tenant_ctx: &crate::tenant::TenantRequestContext, +) -> anyhow::Result<(std::sync::Arc, TimelineRequestContext)> { + let (tline, timeline_ctx) = + tenant.create_empty_timeline(timeline_id, Lsn(8), pg_version, tenant_ctx)?; + let tline = tline.initialize(&timeline_ctx)?; let mut m = tline.begin_modification(Lsn(8)); m.init_empty()?; m.commit()?; - Ok(tline) + Ok((tline, timeline_ctx)) } #[allow(clippy::bool_assert_comparison)] diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 66f7e79877..9340cdf07d 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -34,6 +34,7 @@ use std::fs::OpenOptions; use std::io; use std::io::Write; use std::ops::Bound::Included; +use std::ops::Deref; use std::path::Path; use std::path::PathBuf; use std::process::Command; @@ -46,6 +47,7 @@ use std::time::{Duration, Instant}; use self::metadata::TimelineMetadata; use self::remote_timeline_client::RemoteTimelineClient; use crate::config::PageServerConf; +use crate::context::RequestContext; use crate::import_datadir; use crate::is_uninit_mark; use crate::metrics::{remove_tenant_metrics, STORAGE_TIME}; @@ -92,7 +94,7 @@ mod timeline; pub mod size; -pub use timeline::{PageReconstructError, Timeline}; +pub use timeline::{PageReconstructError, Timeline, TimelineRequestContext}; // re-export this function so that page_cache.rs can use it. pub use crate::tenant::ephemeral_file::writeback as writeback_ephemeral_file; @@ -171,9 +173,12 @@ impl UninitializedTimeline<'_> { /// /// The new timeline is initialized in Active state, and its background jobs are /// started - pub fn initialize(self) -> anyhow::Result> { + pub fn initialize( + self, + timeline_ctx: &TimelineRequestContext, + ) -> anyhow::Result> { let mut timelines = self.owning_tenant.timelines.lock().unwrap(); - self.initialize_with_lock(&mut timelines, true, true) + self.initialize_with_lock(&mut timelines, true, true, timeline_ctx) } /// Like `initialize`, but the caller is already holding lock on Tenant::timelines. @@ -186,6 +191,7 @@ impl UninitializedTimeline<'_> { timelines: &mut HashMap>, load_layer_map: bool, launch_wal_receiver: bool, + timeline_ctx: &TimelineRequestContext, ) -> anyhow::Result> { let timeline_id = self.timeline_id; let tenant_id = self.owning_tenant.tenant_id; @@ -221,10 +227,10 @@ impl UninitializedTimeline<'_> { new_timeline.set_state(TimelineState::Active); v.insert(Arc::clone(&new_timeline)); - new_timeline.maybe_spawn_flush_loop(); + new_timeline.maybe_spawn_flush_loop(timeline_ctx); if launch_wal_receiver { - new_timeline.launch_wal_receiver(); + new_timeline.launch_wal_receiver(timeline_ctx); } } } @@ -237,18 +243,19 @@ impl UninitializedTimeline<'_> { self, copyin_stream: &mut (impl Stream> + Sync + Send + Unpin), base_lsn: Lsn, + ctx: &TimelineRequestContext, ) -> anyhow::Result> { let raw_timeline = self.raw_timeline()?; let mut reader = tokio_util::io::StreamReader::new(copyin_stream); - import_datadir::import_basebackup_from_tar(raw_timeline, &mut reader, base_lsn) + import_datadir::import_basebackup_from_tar(raw_timeline, &mut reader, base_lsn, ctx) .await .context("Failed to import basebackup")?; // Flush loop needs to be spawned in order to be able to flush. // We want to run proper checkpoint before we mark timeline as available to outside world // Thus spawning flush loop manually and skipping flush_loop setup in initialize_with_lock - raw_timeline.maybe_spawn_flush_loop(); + raw_timeline.maybe_spawn_flush_loop(ctx); fail::fail_point!("before-checkpoint-new-timeline", |_| { bail!("failpoint before-checkpoint-new-timeline"); @@ -259,9 +266,7 @@ impl UninitializedTimeline<'_> { .await .context("Failed to flush after basebackup import")?; - let timeline = self.initialize()?; - - Ok(timeline) + self.initialize(ctx) } fn raw_timeline(&self) -> anyhow::Result<&Arc> { @@ -439,6 +444,7 @@ struct RemoteStartupData { impl Tenant { /// Yet another helper for timeline initialization. /// Contains common part for `load_local_timeline` and `load_remote_timeline` + #[allow(clippy::too_many_arguments)] async fn setup_timeline( &self, timeline_id: TimelineId, @@ -447,6 +453,7 @@ impl Tenant { local_metadata: Option, ancestor: Option>, first_save: bool, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result<()> { let tenant_id = self.tenant_id; @@ -457,7 +464,7 @@ impl Tenant { .context("merge_local_remote_metadata")? .to_owned(); - let timeline = { + let (timeline, _timeline_ctx) = { // avoiding holding it across awaits let mut timelines_accessor = self.timelines.lock().unwrap(); if timelines_accessor.contains_key(&timeline_id) { @@ -472,6 +479,7 @@ impl Tenant { ancestor.clone(), remote_client, )?; + let timeline_ctx = dummy_timeline.get_context(); let timeline = UninitializedTimeline { owning_tenant: self, @@ -481,10 +489,11 @@ impl Tenant { // Do not start walreceiver here. We do need loaded layer map for reconcile_with_remote // But we shouldnt start walreceiver before we have all the data locally, because working walreceiver // will ingest data which may require looking at the layers which are not yet available locally - match timeline.initialize_with_lock(&mut timelines_accessor, true, false) { + match timeline.initialize_with_lock(&mut timelines_accessor, true, false, &timeline_ctx) + { Ok(initialized_timeline) => { timelines_accessor.insert(timeline_id, initialized_timeline.clone()); - Ok(initialized_timeline) + Ok((initialized_timeline, timeline_ctx)) } Err(e) => { error!("Failed to initialize timeline {tenant_id}/{timeline_id}: {e:?}"); @@ -516,10 +525,12 @@ impl Tenant { // Reconcile local state with remote storage, downloading anything that's // missing locally, and scheduling uploads for anything that's missing // in remote storage. + let timeline_ctx = timeline.get_context(); timeline .reconcile_with_remote( up_to_date_metadata, remote_startup_data.as_ref().map(|r| &r.index_part), + &timeline_ctx, ) .await .context("failed to reconcile with remote")? @@ -582,6 +593,8 @@ impl Tenant { Some(remote_storage), )); + let tenant_ctx = tenant.get_context(); + // Do all the hard work in the background let tenant_clone = Arc::clone(&tenant); @@ -593,7 +606,7 @@ impl Tenant { "attach tenant", false, async move { - match tenant_clone.attach().await { + match tenant_clone.attach(&tenant_ctx).await { Ok(_) => {} Err(e) => { tenant_clone.set_broken(&e.to_string()); @@ -609,8 +622,8 @@ impl Tenant { /// /// Background task that downloads all data for a tenant and brings it to Active state. /// - #[instrument(skip(self), fields(tenant_id=%self.tenant_id))] - async fn attach(self: &Arc) -> anyhow::Result<()> { + #[instrument(skip(self, tenant_ctx), fields(tenant_id=%self.tenant_id))] + async fn attach(self: &Arc, tenant_ctx: &TenantRequestContext) -> anyhow::Result<()> { // Create directory with marker file to indicate attaching state. // The load_local_tenants() function in tenant::mgr relies on the marker file // to determine whether a tenant has finished attaching. @@ -710,6 +723,7 @@ impl Tenant { index_parts.remove(&timeline_id).unwrap(), remote_metadata, remote_clients.remove(&timeline_id).unwrap(), + tenant_ctx, ) .await .with_context(|| { @@ -729,7 +743,7 @@ impl Tenant { // Start background operations and open the tenant for business. // The loops will shut themselves down when they notice that the tenant is inactive. - self.activate()?; + self.activate(tenant_ctx)?; info!("Done"); @@ -759,6 +773,7 @@ impl Tenant { index_part: IndexPart, remote_metadata: TimelineMetadata, remote_client: RemoteTimelineClient, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result<()> { info!("downloading index file for timeline {}", timeline_id); tokio::fs::create_dir_all(self.conf.timeline_path(&timeline_id, &self.tenant_id)) @@ -793,6 +808,7 @@ impl Tenant { local_metadata, ancestor, true, + tenant_ctx, ) .await } @@ -845,6 +861,7 @@ impl Tenant { remote_storage, ); let tenant = Arc::new(tenant); + let tenant_ctx = tenant.get_context(); // Do all the hard work in a background task let tenant_clone = Arc::clone(&tenant); @@ -857,7 +874,7 @@ impl Tenant { "initial tenant load", false, async move { - match tenant_clone.load().await { + match tenant_clone.load(&tenant_ctx).await { Ok(()) => {} Err(err) => { tenant_clone.set_broken(&err.to_string()); @@ -878,10 +895,9 @@ impl Tenant { /// Background task to load in-memory data structures for this tenant, from /// files on disk. Used at pageserver startup. /// - #[instrument(skip(self), fields(tenant_id=%self.tenant_id))] - async fn load(self: &Arc) -> anyhow::Result<()> { + #[instrument(skip(self, tenant_ctx), fields(tenant_id=%self.tenant_id))] + async fn load(self: &Arc, tenant_ctx: &TenantRequestContext) -> anyhow::Result<()> { info!("loading tenant task"); - utils::failpoint_sleep_millis_async!("before-loading-tenant"); // TODO split this into two functions, scan and actual load @@ -990,14 +1006,14 @@ impl Tenant { // 1. "Timeline has no ancestor and no layer files" for (timeline_id, local_metadata) in sorted_timelines { - self.load_local_timeline(timeline_id, local_metadata) + self.load_local_timeline(timeline_id, local_metadata, tenant_ctx) .await .with_context(|| format!("load local timeline {timeline_id}"))?; } // Start background operations and open the tenant for business. // The loops will shut themselves down when they notice that the tenant is inactive. - self.activate()?; + self.activate(tenant_ctx)?; info!("Done"); @@ -1007,11 +1023,12 @@ impl Tenant { /// Subroutine of `load_tenant`, to load an individual timeline /// /// NB: The parent is assumed to be already loaded! - #[instrument(skip(self, local_metadata), fields(timeline_id=%timeline_id))] + #[instrument(skip(self, local_metadata, tenant_ctx), fields(timeline_id=%timeline_id))] async fn load_local_timeline( &self, timeline_id: TimelineId, local_metadata: TimelineMetadata, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result<()> { let ancestor = if let Some(ancestor_timeline_id) = local_metadata.ancestor_timeline() { let ancestor_timeline = self.get_timeline(ancestor_timeline_id, false) @@ -1055,6 +1072,7 @@ impl Tenant { Some(local_metadata), ancestor, false, + tenant_ctx, ) .await } @@ -1101,12 +1119,13 @@ impl Tenant { /// This is used to create the initial 'main' timeline during bootstrapping, /// or when importing a new base backup. The caller is expected to load an /// initial image of the datadir to the new timeline after this. - pub fn create_empty_timeline( + pub fn create_empty_timeline<'a>( &self, new_timeline_id: TimelineId, initdb_lsn: Lsn, pg_version: u32, - ) -> anyhow::Result { + tenant_ctx: &'a TenantRequestContext, + ) -> anyhow::Result<(UninitializedTimeline, TimelineRequestContext)> { anyhow::ensure!( self.is_active(), "Cannot create empty timelines on inactive tenant" @@ -1125,13 +1144,16 @@ impl Tenant { initdb_lsn, pg_version, ); - self.prepare_timeline( + let timeline = self.prepare_timeline( new_timeline_id, new_metadata, timeline_uninit_mark, true, None, - ) + tenant_ctx, + )?; + let timeline_ctx = timeline.raw_timeline().unwrap().get_context(); + Ok((timeline, timeline_ctx)) } /// Create a new timeline. @@ -1147,6 +1169,7 @@ impl Tenant { ancestor_timeline_id: Option, mut ancestor_start_lsn: Option, pg_version: u32, + ctx: &TenantRequestContext, ) -> anyhow::Result>> { anyhow::ensure!( self.is_active(), @@ -1163,6 +1186,7 @@ impl Tenant { let ancestor_timeline = self .get_timeline(ancestor_timeline_id, false) .context("Cannot branch off the timeline that's not present in pageserver")?; + let ancestor_ctx = ancestor_timeline.get_context(); if let Some(lsn) = ancestor_start_lsn.as_mut() { *lsn = lsn.align(); @@ -1184,13 +1208,16 @@ impl Tenant { // decoding the new WAL might need to look up previous pages, relation // sizes etc. and that would get confused if the previous page versions // are not in the repository yet. - ancestor_timeline.wait_lsn(*lsn).await?; + ancestor_timeline.wait_lsn(*lsn, &ancestor_ctx).await?; } - self.branch_timeline(ancestor_timeline_id, new_timeline_id, ancestor_start_lsn) + self.branch_timeline(&ancestor_timeline, new_timeline_id, ancestor_start_lsn, ctx) + .await? + } + None => { + self.bootstrap_timeline(new_timeline_id, pg_version, ctx) .await? } - None => self.bootstrap_timeline(new_timeline_id, pg_version).await?, }; Ok(Some(loaded_timeline)) @@ -1214,6 +1241,7 @@ impl Tenant { target_timeline_id: Option, horizon: u64, pitr: Duration, + ctx: &TenantRequestContext, ) -> anyhow::Result { anyhow::ensure!( self.is_active(), @@ -1228,7 +1256,7 @@ impl Tenant { let _timer = STORAGE_TIME .with_label_values(&["gc", &self.tenant_id.to_string(), &timeline_str]) .start_timer(); - self.gc_iteration_internal(target_timeline_id, horizon, pitr) + self.gc_iteration_internal(target_timeline_id, horizon, pitr, ctx) .await } } @@ -1237,7 +1265,10 @@ impl Tenant { /// This function is periodically called by compactor task. /// Also it can be explicitly requested per timeline through page server /// api's 'compact' command. - pub async fn compaction_iteration(&self) -> anyhow::Result<()> { + pub async fn compaction_iteration( + &self, + tenant_ctx: &TenantRequestContext, + ) -> anyhow::Result<()> { anyhow::ensure!( self.is_active(), "Cannot run compaction iteration on inactive tenant" @@ -1258,8 +1289,9 @@ impl Tenant { }; for (timeline_id, timeline) in &timelines_to_compact { + let timeline_ctx = timeline.get_context(); timeline - .compact() + .compact(&timeline_ctx) .instrument(info_span!("compact_timeline", timeline = %timeline_id)) .await?; } @@ -1292,7 +1324,11 @@ impl Tenant { } /// Removes timeline-related in-memory data - pub async fn delete_timeline(&self, timeline_id: TimelineId) -> anyhow::Result<()> { + pub async fn delete_timeline( + &self, + timeline_id: TimelineId, + _ctx: &TenantRequestContext, + ) -> anyhow::Result<()> { // Transition the timeline into TimelineState::Stopping. // This should prevent new operations from starting. let timeline = { @@ -1422,7 +1458,7 @@ impl Tenant { } /// Changes tenant status to active, unless shutdown was already requested. - fn activate(&self) -> anyhow::Result<()> { + fn activate(self: &Arc, tenant_ctx: &TenantRequestContext) -> anyhow::Result<()> { let mut result = Ok(()); self.state.send_modify(|current_state| { match *current_state { @@ -1456,8 +1492,9 @@ impl Tenant { tasks::start_background_loops(self.tenant_id); for timeline in not_broken_timelines { + let timeline_ctx = timeline.get_context(); timeline.set_state(TimelineState::Active); - timeline.launch_wal_receiver(); + timeline.launch_wal_receiver(&timeline_ctx); } } } @@ -1531,7 +1568,20 @@ impl Tenant { self.state.subscribe() } - pub async fn wait_to_become_active(&self) -> anyhow::Result<()> { + /// Return a new TenantRequestContext. + /// + /// XXX: This is a placeholder. In the next commit, this will + /// check that the tenant is still active. + pub fn get_context(self: &Arc) -> TenantRequestContext { + TenantRequestContext { + ctx: RequestContext {}, + } + } + + pub async fn wait_to_become_active( + self: &Arc, + ctx: &RequestContext, + ) -> anyhow::Result<()> { let mut receiver = self.state.subscribe(); loop { let current_state = *receiver.borrow_and_update(); @@ -1864,12 +1914,13 @@ impl Tenant { target_timeline_id: Option, horizon: u64, pitr: Duration, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result { let mut totals: GcResult = Default::default(); let now = Instant::now(); let gc_timelines = self - .refresh_gc_info_internal(target_timeline_id, horizon, pitr) + .refresh_gc_info_internal(target_timeline_id, horizon, pitr, tenant_ctx) .await?; utils::failpoint_sleep_millis_async!("gc_iteration_internal_after_getting_gc_timelines"); @@ -1897,7 +1948,9 @@ impl Tenant { // made. break; } - let result = timeline.gc().await?; + + let timeline_ctx = timeline.get_context(); + let result = timeline.gc(&timeline_ctx).await?; totals += result; } @@ -1910,7 +1963,10 @@ impl Tenant { /// [`Tenant::get_gc_horizon`]. /// /// This is usually executed as part of periodic gc, but can now be triggered more often. - pub async fn refresh_gc_info(&self) -> anyhow::Result>> { + pub async fn refresh_gc_info( + &self, + ctx: &TenantRequestContext, + ) -> anyhow::Result>> { // since this method can now be called at different rates than the configured gc loop, it // might be that these configuration values get applied faster than what it was previously, // since these were only read from the gc task. @@ -1920,7 +1976,7 @@ impl Tenant { // refresh all timelines let target_timeline_id = None; - self.refresh_gc_info_internal(target_timeline_id, horizon, pitr) + self.refresh_gc_info_internal(target_timeline_id, horizon, pitr, ctx) .await } @@ -1929,6 +1985,7 @@ impl Tenant { target_timeline_id: Option, horizon: u64, pitr: Duration, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result>> { // grab mutex to prevent new timelines from being created here. let gc_cs = self.gc_cs.lock().await; @@ -1984,6 +2041,7 @@ impl Tenant { let timeline = self .get_timeline(timeline_id, false) .with_context(|| format!("Timeline {timeline_id} was not found"))?; + let timeline_ctx = timeline.get_context(); // If target_timeline is specified, ignore all other timelines if let Some(target_timeline_id) = target_timeline_id { @@ -2000,7 +2058,9 @@ impl Tenant { )) .map(|&x| x.1) .collect(); - timeline.update_gc_info(branchpoints, cutoff, pitr).await?; + timeline + .update_gc_info(branchpoints, cutoff, pitr, &timeline_ctx) + .await?; gc_timelines.push(timeline); } @@ -2012,10 +2072,13 @@ impl Tenant { /// Branch an existing timeline async fn branch_timeline( &self, - src: TimelineId, + src_timeline: &Arc, dst: TimelineId, start_lsn: Option, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result> { + let src = src_timeline.timeline_id; + // We need to hold this lock to prevent GC from starting at the same time. GC scans the directory to learn // about timelines, so otherwise a race condition is possible, where we create new timeline and GC // concurrently removes data that is needed by the new timeline. @@ -2034,13 +2097,6 @@ impl Tenant { // Step 2 is to avoid initializing the new branch using data removed by past GC iterations // or in-queue GC iterations. - let src_timeline = self.get_timeline(src, false).with_context(|| { - format!( - "No ancestor {} found for timeline {}/{}", - src, self.tenant_id, dst - ) - })?; - let latest_gc_cutoff_lsn = src_timeline.get_latest_gc_cutoff_lsn(); // If no start LSN is specified, we branch the new timeline from the source timeline's last record LSN @@ -2093,16 +2149,19 @@ impl Tenant { src_timeline.initdb_lsn, src_timeline.pg_version, ); + let mut timelines = self.timelines.lock().unwrap(); - let new_timeline = self - .prepare_timeline( - dst, - metadata, - timeline_uninit_mark, - false, - Some(src_timeline), - )? - .initialize_with_lock(&mut timelines, true, true)?; + let uninit_timeline = self.prepare_timeline( + dst, + metadata, + timeline_uninit_mark, + false, + Some(Arc::clone(src_timeline)), + tenant_ctx, + )?; + let new_timeline_ctx = uninit_timeline.raw_timeline().unwrap().get_context(); + let new_timeline = + uninit_timeline.initialize_with_lock(&mut timelines, true, true, &new_timeline_ctx)?; drop(timelines); info!("branched timeline {dst} from {src} at {start_lsn}"); @@ -2115,6 +2174,7 @@ impl Tenant { &self, timeline_id: TimelineId, pg_version: u32, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result> { let timeline_uninit_mark = { let timelines = self.timelines.lock().unwrap(); @@ -2164,16 +2224,24 @@ impl Tenant { pgdata_lsn, pg_version, ); - let raw_timeline = - self.prepare_timeline(timeline_id, new_metadata, timeline_uninit_mark, true, None)?; + let raw_timeline = self.prepare_timeline( + timeline_id, + new_metadata, + timeline_uninit_mark, + true, + None, + tenant_ctx, + )?; let tenant_id = raw_timeline.owning_tenant.tenant_id; let unfinished_timeline = raw_timeline.raw_timeline()?; + let timeline_ctx = unfinished_timeline.get_context(); import_datadir::import_timeline_from_postgres_datadir( unfinished_timeline, pgdata_path, pgdata_lsn, + &timeline_ctx, ) .await .with_context(|| { @@ -2184,7 +2252,7 @@ impl Tenant { // the outside world. // // Thus spawn flush loop manually and skip flush_loop setup in initialize_with_lock - unfinished_timeline.maybe_spawn_flush_loop(); + unfinished_timeline.maybe_spawn_flush_loop(&timeline_ctx); fail::fail_point!("before-checkpoint-new-timeline", |_| { anyhow::bail!("failpoint before-checkpoint-new-timeline"); @@ -2201,7 +2269,7 @@ impl Tenant { let timeline = { let mut timelines = self.timelines.lock().unwrap(); - raw_timeline.initialize_with_lock(&mut timelines, false, true)? + raw_timeline.initialize_with_lock(&mut timelines, false, true, &timeline_ctx)? }; info!( @@ -2222,6 +2290,7 @@ impl Tenant { uninit_mark: TimelineUninitMark, init_layers: bool, ancestor: Option>, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result { let tenant_id = self.tenant_id; @@ -2241,7 +2310,7 @@ impl Tenant { match self.create_timeline_files( &uninit_mark.timeline_path, new_timeline_id, - new_metadata, + &new_metadata, ancestor, remote_client, ) { @@ -2271,7 +2340,7 @@ impl Tenant { &self, timeline_path: &Path, new_timeline_id: TimelineId, - new_metadata: TimelineMetadata, + new_metadata: &TimelineMetadata, ancestor: Option>, remote_client: Option, ) -> anyhow::Result> { @@ -2293,7 +2362,7 @@ impl Tenant { self.conf, new_timeline_id, self.tenant_id, - &new_metadata, + new_metadata, true, ) .context("Failed to create timeline metadata")?; @@ -2345,7 +2414,10 @@ impl Tenant { /// /// Future is cancellation safe. Only one calculation can be running at once per tenant. #[instrument(skip_all, fields(tenant_id=%self.tenant_id))] - pub async fn gather_size_inputs(&self) -> anyhow::Result { + pub async fn gather_size_inputs( + &self, + ctx: &TenantRequestContext, + ) -> anyhow::Result { let logical_sizes_at_once = self .conf .concurrent_tenant_size_logical_size_queries @@ -2357,7 +2429,7 @@ impl Tenant { // See more for on the issue #2748 condenced out of the initial PR review. let mut shared_cache = self.cached_logical_sizes.lock().await; - size::gather_inputs(self, logical_sizes_at_once, &mut shared_cache).await + size::gather_inputs(self, logical_sizes_at_once, &mut shared_cache, ctx).await } } @@ -2566,6 +2638,20 @@ impl Drop for Tenant { remove_tenant_metrics(&self.tenant_id); } } + +/// XXX: Placeholder +pub struct TenantRequestContext { + ctx: RequestContext, +} + +impl Deref for TenantRequestContext { + type Target = RequestContext; + + fn deref(&self) -> &RequestContext { + &self.ctx + } +} + /// Dump contents of a layer file to stdout. pub fn dump_layerfile_from_path(path: &Path, verbose: bool) -> anyhow::Result<()> { use std::os::unix::fs::FileExt; @@ -2665,6 +2751,8 @@ pub mod harness { Option>, Option>, ), + + pub test_ctx: RequestContext, } impl<'a> TenantHarness<'a> { @@ -2702,11 +2790,14 @@ pub mod harness { fs::create_dir_all(conf.tenant_path(&tenant_id))?; fs::create_dir_all(conf.timelines_path(&tenant_id))?; + let test_ctx = RequestContext::new(); + Ok(Self { conf, tenant_conf, tenant_id, lock_guard, + test_ctx, }) } @@ -2725,6 +2816,7 @@ pub mod harness { self.tenant_id, None, )); + let tenant_ctx = tenant.get_context(); // populate tenant with locally available timelines let mut timelines_to_load = HashMap::new(); for timeline_dir_entry in fs::read_dir(self.conf.timelines_path(&self.tenant_id)) @@ -2742,8 +2834,7 @@ pub mod harness { timelines_to_load.insert(timeline_id, timeline_metadata); } // FIXME starts background jobs - tenant.load().await?; - + tenant.load(&tenant_ctx).await?; Ok(tenant) } @@ -2801,9 +2892,10 @@ mod tests { #[tokio::test] async fn test_basic() -> anyhow::Result<()> { let tenant = TenantHarness::create("test_basic")?.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&ctx)?; let writer = tline.writer(); writer.put(*TEST_KEY, Lsn(0x10), &Value::Image(TEST_IMG("foo at 0x10")))?; @@ -2816,15 +2908,15 @@ mod tests { drop(writer); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x10)).await?, + tline.get(*TEST_KEY, Lsn(0x10), &ctx).await?, TEST_IMG("foo at 0x10") ); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x1f)).await?, + tline.get(*TEST_KEY, Lsn(0x1f), &ctx).await?, TEST_IMG("foo at 0x10") ); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x20)).await?, + tline.get(*TEST_KEY, Lsn(0x20), &ctx).await?, TEST_IMG("foo at 0x20") ); @@ -2836,11 +2928,12 @@ mod tests { let tenant = TenantHarness::create("no_duplicate_timelines")? .load() .await; - let _ = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (timeline, timeline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let _ = timeline.initialize(&timeline_ctx)?; - match tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION) { + match tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx) { Ok(_) => panic!("duplicate timeline creation should fail"), Err(e) => assert_eq!( e.to_string(), @@ -2866,13 +2959,15 @@ mod tests { /// #[tokio::test] async fn test_branch() -> anyhow::Result<()> { - let tenant = TenantHarness::create("test_branch")?.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; - let writer = tline.writer(); use std::str::from_utf8; + let tenant = TenantHarness::create("test_branch")?.load().await; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; + let writer = tline.writer(); + #[allow(non_snake_case)] let TEST_KEY_A: Key = Key::from_hex("112222222233333333444444445500000001").unwrap(); #[allow(non_snake_case)] @@ -2891,27 +2986,25 @@ mod tests { //assert_current_logical_size(&tline, Lsn(0x40)); // Branch the history, modify relation differently on the new timeline - tenant - .branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x30))) - .await?; let newtline = tenant - .get_timeline(NEW_TIMELINE_ID, true) - .expect("Should have a local timeline"); + .branch_timeline(&tline, NEW_TIMELINE_ID, Some(Lsn(0x30)), &tenant_ctx) + .await?; + let newtline_ctx = newtline.get_context(); let new_writer = newtline.writer(); new_writer.put(TEST_KEY_A, Lsn(0x40), &test_value("bar at 0x40"))?; new_writer.finish_write(Lsn(0x40)); // Check page contents on both branches assert_eq!( - from_utf8(&tline.get(TEST_KEY_A, Lsn(0x40)).await?)?, + from_utf8(&tline.get(TEST_KEY_A, Lsn(0x40), &tline_ctx).await?)?, "foo at 0x40" ); assert_eq!( - from_utf8(&newtline.get(TEST_KEY_A, Lsn(0x40)).await?)?, + from_utf8(&newtline.get(TEST_KEY_A, Lsn(0x40), &newtline_ctx).await?)?, "bar at 0x40" ); assert_eq!( - from_utf8(&newtline.get(TEST_KEY_B, Lsn(0x40)).await?)?, + from_utf8(&newtline.get(TEST_KEY_B, Lsn(0x40), &newtline_ctx).await?)?, "foobar at 0x20" ); @@ -2967,9 +3060,10 @@ mod tests { TenantHarness::create("test_prohibit_branch_creation_on_garbage_collected_data")? .load() .await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; make_some_layers(tline.as_ref(), Lsn(0x20)).await?; // this removes layers before lsn 40 (50 minus 10), so there are two remaining layers, image and delta for 31-50 @@ -2977,12 +3071,12 @@ mod tests { // and compaction works. But it does set the 'cutoff' point so that the cross check // below should fail. tenant - .gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO) + .gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO, &tenant_ctx) .await?; // try to branch at lsn 25, should fail because we already garbage collected the data match tenant - .branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x25))) + .branch_timeline(&tline, NEW_TIMELINE_ID, Some(Lsn(0x25)), &tenant_ctx) .await { Ok(_) => panic!("branching should have failed"), @@ -3004,13 +3098,17 @@ mod tests { let tenant = TenantHarness::create("test_prohibit_branch_creation_on_pre_initdb_lsn")? .load() .await; - - tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0x50), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = tenant.create_empty_timeline( + TIMELINE_ID, + Lsn(0x50), + DEFAULT_PG_VERSION, + &tenant_ctx, + )?; + let tline = tline.initialize(&tline_ctx)?; // try to branch at lsn 0x25, should fail because initdb lsn is 0x50 match tenant - .branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x25))) + .branch_timeline(&tline, NEW_TIMELINE_ID, Some(Lsn(0x25)), &tenant_ctx) .await { Ok(_) => panic!("branching should have failed"), @@ -3055,22 +3153,27 @@ mod tests { let tenant = TenantHarness::create("test_retain_data_in_parent_which_is_needed_for_child")? .load() .await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; make_some_layers(tline.as_ref(), Lsn(0x20)).await?; tenant - .branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40))) + .branch_timeline(&tline, NEW_TIMELINE_ID, Some(Lsn(0x40)), &tenant_ctx) .await?; let newtline = tenant .get_timeline(NEW_TIMELINE_ID, true) .expect("Should have a local timeline"); + let newtline_ctx = newtline.get_context(); // this removes layers before lsn 40 (50 minus 10), so there are two remaining layers, image and delta for 31-50 tenant - .gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO) + .gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO, &tenant_ctx) .await?; - assert!(newtline.get(*TEST_KEY, Lsn(0x25)).await.is_ok()); + assert!(newtline + .get(*TEST_KEY, Lsn(0x25), &newtline_ctx) + .await + .is_ok()); Ok(()) } @@ -3079,28 +3182,30 @@ mod tests { let tenant = TenantHarness::create("test_parent_keeps_data_forever_after_branching")? .load() .await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; make_some_layers(tline.as_ref(), Lsn(0x20)).await?; tenant - .branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40))) + .branch_timeline(&tline, NEW_TIMELINE_ID, Some(Lsn(0x40)), &tenant_ctx) .await?; let newtline = tenant .get_timeline(NEW_TIMELINE_ID, true) .expect("Should have a local timeline"); + let newtline_ctx = newtline.get_context(); make_some_layers(newtline.as_ref(), Lsn(0x60)).await?; // run gc on parent tenant - .gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO) + .gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO, &tenant_ctx) .await?; // Check that the data is still accessible on the branch. assert_eq!( - newtline.get(*TEST_KEY, Lsn(0x50)).await?, + newtline.get(*TEST_KEY, Lsn(0x50), &newtline_ctx).await?, TEST_IMG(&format!("foo at {}", Lsn(0x40))) ); @@ -3113,9 +3218,14 @@ mod tests { let harness = TenantHarness::create(TEST_NAME)?; { let tenant = harness.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0x8000), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = tenant.create_empty_timeline( + TIMELINE_ID, + Lsn(0x8000), + DEFAULT_PG_VERSION, + &tenant_ctx, + )?; + let tline = tline.initialize(&tline_ctx)?; make_some_layers(tline.as_ref(), Lsn(0x8000)).await?; } @@ -3134,14 +3244,19 @@ mod tests { // create two timelines { let tenant = harness.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = tenant.create_empty_timeline( + TIMELINE_ID, + Lsn(0), + DEFAULT_PG_VERSION, + &tenant_ctx, + )?; + let tline = tline.initialize(&tline_ctx)?; make_some_layers(tline.as_ref(), Lsn(0x20)).await?; tenant - .branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40))) + .branch_timeline(&tline, NEW_TIMELINE_ID, Some(Lsn(0x40)), &tenant_ctx) .await?; let newtline = tenant @@ -3170,12 +3285,19 @@ mod tests { async fn corrupt_metadata() -> anyhow::Result<()> { const TEST_NAME: &str = "corrupt_metadata"; let harness = TenantHarness::create(TEST_NAME)?; - let tenant = harness.load().await; - tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; - drop(tenant); + { + let tenant = harness.load().await; + let tenant_ctx = tenant.get_context(); + + let (tline, tline_ctx) = tenant.create_empty_timeline( + TIMELINE_ID, + Lsn(0), + DEFAULT_PG_VERSION, + &tenant_ctx, + )?; + let _ = tline.initialize(&tline_ctx)?; + } let metadata_path = harness.timeline_path(&TIMELINE_ID).join(METADATA_FILE_NAME); @@ -3211,9 +3333,10 @@ mod tests { #[tokio::test] async fn test_images() -> anyhow::Result<()> { let tenant = TenantHarness::create("test_images")?.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; let writer = tline.writer(); writer.put(*TEST_KEY, Lsn(0x10), &Value::Image(TEST_IMG("foo at 0x10")))?; @@ -3221,7 +3344,7 @@ mod tests { drop(writer); tline.freeze_and_flush().await?; - tline.compact().await?; + tline.compact(&tline_ctx).await?; let writer = tline.writer(); writer.put(*TEST_KEY, Lsn(0x20), &Value::Image(TEST_IMG("foo at 0x20")))?; @@ -3229,7 +3352,7 @@ mod tests { drop(writer); tline.freeze_and_flush().await?; - tline.compact().await?; + tline.compact(&tline_ctx).await?; let writer = tline.writer(); writer.put(*TEST_KEY, Lsn(0x30), &Value::Image(TEST_IMG("foo at 0x30")))?; @@ -3237,7 +3360,7 @@ mod tests { drop(writer); tline.freeze_and_flush().await?; - tline.compact().await?; + tline.compact(&tline_ctx).await?; let writer = tline.writer(); writer.put(*TEST_KEY, Lsn(0x40), &Value::Image(TEST_IMG("foo at 0x40")))?; @@ -3245,26 +3368,26 @@ mod tests { drop(writer); tline.freeze_and_flush().await?; - tline.compact().await?; + tline.compact(&tline_ctx).await?; assert_eq!( - tline.get(*TEST_KEY, Lsn(0x10)).await?, + tline.get(*TEST_KEY, Lsn(0x10), &tline_ctx).await?, TEST_IMG("foo at 0x10") ); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x1f)).await?, + tline.get(*TEST_KEY, Lsn(0x1f), &tline_ctx).await?, TEST_IMG("foo at 0x10") ); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x20)).await?, + tline.get(*TEST_KEY, Lsn(0x20), &tline_ctx).await?, TEST_IMG("foo at 0x20") ); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x30)).await?, + tline.get(*TEST_KEY, Lsn(0x30), &tline_ctx).await?, TEST_IMG("foo at 0x30") ); assert_eq!( - tline.get(*TEST_KEY, Lsn(0x40)).await?, + tline.get(*TEST_KEY, Lsn(0x40), &tline_ctx).await?, TEST_IMG("foo at 0x40") ); @@ -3278,9 +3401,10 @@ mod tests { #[tokio::test] async fn test_bulk_insert() -> anyhow::Result<()> { let tenant = TenantHarness::create("test_bulk_insert")?.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; let mut lsn = Lsn(0x10); @@ -3309,11 +3433,11 @@ mod tests { let cutoff = tline.get_last_record_lsn(); tline - .update_gc_info(Vec::new(), cutoff, Duration::ZERO) + .update_gc_info(Vec::new(), cutoff, Duration::ZERO, &tline_ctx) .await?; tline.freeze_and_flush().await?; - tline.compact().await?; - tline.gc().await?; + tline.compact(&tline_ctx).await?; + tline.gc(&tline_ctx).await?; } Ok(()) @@ -3322,9 +3446,10 @@ mod tests { #[tokio::test] async fn test_random_updates() -> anyhow::Result<()> { let tenant = TenantHarness::create("test_random_updates")?.load().await; - let tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let tline = tline.initialize(&tline_ctx)?; const NUM_KEYS: usize = 1000; @@ -3374,7 +3499,7 @@ mod tests { for (blknum, last_lsn) in updated.iter().enumerate() { test_key.field6 = blknum as u32; assert_eq!( - tline.get(test_key, lsn).await?, + tline.get(test_key, lsn, &tline_ctx).await?, TEST_IMG(&format!("{} at {}", blknum, last_lsn)) ); } @@ -3382,11 +3507,11 @@ mod tests { // Perform a cycle of flush, compact, and GC let cutoff = tline.get_last_record_lsn(); tline - .update_gc_info(Vec::new(), cutoff, Duration::ZERO) + .update_gc_info(Vec::new(), cutoff, Duration::ZERO, &tline_ctx) .await?; tline.freeze_and_flush().await?; - tline.compact().await?; - tline.gc().await?; + tline.compact(&tline_ctx).await?; + tline.gc(&tline_ctx).await?; } Ok(()) @@ -3397,9 +3522,10 @@ mod tests { let tenant = TenantHarness::create("test_traverse_branches")? .load() .await; - let mut tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (init_tline, init_tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let init_tline = init_tline.initialize(&init_tline_ctx)?; const NUM_KEYS: usize = 1000; @@ -3416,7 +3542,7 @@ mod tests { for blknum in 0..NUM_KEYS { lsn = Lsn(lsn.0 + 0x10); test_key.field6 = blknum as u32; - let writer = tline.writer(); + let writer = init_tline.writer(); writer.put( test_key, lsn, @@ -3429,16 +3555,17 @@ mod tests { keyspace.add_key(test_key); } - let mut tline_id = TIMELINE_ID; + let mut tline = init_tline; + let mut tline_ctx; for _ in 0..50 { let new_tline_id = TimelineId::generate(); tenant - .branch_timeline(tline_id, new_tline_id, Some(lsn)) + .branch_timeline(&tline, new_tline_id, Some(lsn), &tenant_ctx) .await?; tline = tenant .get_timeline(new_tline_id, true) .expect("Should have the branched timeline"); - tline_id = new_tline_id; + tline_ctx = tline.get_context(); for _ in 0..NUM_KEYS { lsn = Lsn(lsn.0 + 0x10); @@ -3460,7 +3587,7 @@ mod tests { for (blknum, last_lsn) in updated.iter().enumerate() { test_key.field6 = blknum as u32; assert_eq!( - tline.get(test_key, lsn).await?, + tline.get(test_key, lsn, &tline_ctx).await?, TEST_IMG(&format!("{} at {}", blknum, last_lsn)) ); } @@ -3468,11 +3595,11 @@ mod tests { // Perform a cycle of flush, compact, and GC let cutoff = tline.get_last_record_lsn(); tline - .update_gc_info(Vec::new(), cutoff, Duration::ZERO) + .update_gc_info(Vec::new(), cutoff, Duration::ZERO, &tline_ctx) .await?; tline.freeze_and_flush().await?; - tline.compact().await?; - tline.gc().await?; + tline.compact(&tline_ctx).await?; + tline.gc(&tline_ctx).await?; } Ok(()) @@ -3483,9 +3610,10 @@ mod tests { let tenant = TenantHarness::create("test_traverse_ancestors")? .load() .await; - let mut tline = tenant - .create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)? - .initialize()?; + let tenant_ctx = tenant.get_context(); + let (tline, tline_ctx) = + tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &tenant_ctx)?; + let mut tline = tline.initialize(&tline_ctx)?; const NUM_KEYS: usize = 100; const NUM_TLINES: usize = 50; @@ -3495,18 +3623,16 @@ mod tests { let mut updated = [[Lsn(0); NUM_KEYS]; NUM_TLINES]; let mut lsn = Lsn(0); - let mut tline_id = TIMELINE_ID; #[allow(clippy::needless_range_loop)] for idx in 0..NUM_TLINES { let new_tline_id = TimelineId::generate(); tenant - .branch_timeline(tline_id, new_tline_id, Some(lsn)) + .branch_timeline(&tline, new_tline_id, Some(lsn), &tenant_ctx) .await?; tline = tenant .get_timeline(new_tline_id, true) .expect("Should have the branched timeline"); - tline_id = new_tline_id; for _ in 0..NUM_KEYS { lsn = Lsn(lsn.0 + 0x10); @@ -3535,7 +3661,7 @@ mod tests { println!("checking [{idx}][{blknum}] at {lsn}"); test_key.field6 = blknum as u32; assert_eq!( - tline.get(test_key, *lsn).await?, + tline.get(test_key, *lsn, &tline_ctx).await?, TEST_IMG(&format!("{idx} {blknum} at {lsn}")) ); } diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index dce7cd8bae..deca21f106 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -16,6 +16,7 @@ use remote_storage::GenericRemoteStorage; use utils::crashsafe; use crate::config::PageServerConf; +use crate::context::RequestContext; use crate::task_mgr::{self, TaskKind}; use crate::tenant::config::TenantConfOpt; use crate::tenant::{Tenant, TenantState}; @@ -234,6 +235,7 @@ pub async fn update_tenant_config( conf: &'static PageServerConf, tenant_conf: TenantConfOpt, tenant_id: TenantId, + ctx: &RequestContext, ) -> anyhow::Result<()> { info!("configuring tenant {tenant_id}"); get_tenant(tenant_id, true) @@ -260,10 +262,15 @@ pub async fn get_tenant(tenant_id: TenantId, active_only: bool) -> anyhow::Resul } } -pub async fn delete_timeline(tenant_id: TenantId, timeline_id: TimelineId) -> anyhow::Result<()> { +pub async fn delete_timeline( + tenant_id: TenantId, + timeline_id: TimelineId, + ctx: &RequestContext, +) -> anyhow::Result<()> { match get_tenant(tenant_id, true).await { Ok(tenant) => { - tenant.delete_timeline(timeline_id).await?; + let tenant_ctx = tenant.get_context(); + tenant.delete_timeline(timeline_id, &tenant_ctx).await?; } Err(e) => anyhow::bail!("Cannot access tenant {tenant_id} in local tenant state: {e:?}"), } @@ -473,8 +480,9 @@ pub async fn immediate_gc( false, async move { fail::fail_point!("immediate_gc_task_pre"); + let tenant_ctx = tenant.get_context(); let result = tenant - .gc_iteration(Some(timeline_id), gc_horizon, pitr) + .gc_iteration(Some(timeline_id), gc_horizon, pitr, &tenant_ctx) .instrument(info_span!("manual_gc", tenant = %tenant_id, timeline = %timeline_id)) .await; // FIXME: `gc_iteration` can return an error for multiple reasons; we should handle it @@ -509,6 +517,7 @@ pub async fn immediate_compact( let timeline = tenant .get_timeline(timeline_id, true) .map_err(ApiError::NotFound)?; + let timeline_ctx = timeline.get_context(); // Run in task_mgr to avoid race with detach operation let (task_done, wait_task_done) = tokio::sync::oneshot::channel(); @@ -523,7 +532,7 @@ pub async fn immediate_compact( false, async move { let result = timeline - .compact() + .compact(&timeline_ctx) .instrument( info_span!("manual_compact", tenant = %tenant_id, timeline = %timeline_id), ) diff --git a/pageserver/src/tenant/size.rs b/pageserver/src/tenant/size.rs index aa11985cbe..a298f4d457 100644 --- a/pageserver/src/tenant/size.rs +++ b/pageserver/src/tenant/size.rs @@ -7,6 +7,7 @@ use tokio::sync::oneshot::error::RecvError; use tokio::sync::Semaphore; use crate::pgdatadir_mapping::CalculateLogicalSizeError; +use crate::tenant::{TenantRequestContext, TimelineRequestContext}; use super::Tenant; use utils::id::TimelineId; @@ -63,13 +64,14 @@ pub(super) async fn gather_inputs( tenant: &Tenant, limit: &Arc, logical_size_cache: &mut HashMap<(TimelineId, Lsn), u64>, + tenant_ctx: &TenantRequestContext, ) -> anyhow::Result { // with joinset, on drop, all of the tasks will just be de-scheduled, which we can use to // our advantage with `?` error handling. let mut joinset = tokio::task::JoinSet::new(); let timelines = tenant - .refresh_gc_info() + .refresh_gc_info(tenant_ctx) .await .context("Failed to refresh gc_info before gathering inputs")?; @@ -100,6 +102,9 @@ pub(super) async fn gather_inputs( for timeline in timelines { let last_record_lsn = timeline.get_last_record_lsn(); + let ctx = timeline.get_context(); + let ctx = Arc::new(ctx); + let (interesting_lsns, horizon_cutoff, pitr_cutoff, next_gc_cutoff) = { // there's a race between the update (holding tenant.gc_lock) and this read but it // might not be an issue, because it's not for Timeline::gc @@ -169,19 +174,23 @@ pub(super) async fn gather_inputs( timeline_id: timeline.timeline_id, }); - for (lsn, _kind) in &interesting_lsns { - if let Some(size) = logical_size_cache.get(&(timeline.timeline_id, *lsn)) { + for (lsn, _kind) in interesting_lsns.iter() { + let lsn = *lsn; + if let Some(size) = logical_size_cache.get(&(timeline.timeline_id, lsn)) { updates.push(Update { - lsn: *lsn, + lsn, timeline_id: timeline.timeline_id, command: Command::Update(*size), }); - needed_cache.insert((timeline.timeline_id, *lsn)); + needed_cache.insert((timeline.timeline_id, lsn)); } else { let timeline = Arc::clone(&timeline); let parallel_size_calcs = Arc::clone(limit); - joinset.spawn(calculate_logical_size(parallel_size_calcs, timeline, *lsn)); + let ctx_clone = Arc::clone(&ctx); + joinset.spawn(async move { + calculate_logical_size(parallel_size_calcs, timeline, lsn, &ctx_clone).await + }); } } @@ -365,13 +374,14 @@ async fn calculate_logical_size( limit: Arc, timeline: Arc, lsn: utils::lsn::Lsn, + ctx: &TimelineRequestContext, ) -> Result { let _permit = tokio::sync::Semaphore::acquire_owned(limit) .await - .expect("global semaphore should not had been closed"); + .expect("global semaphore should not have been closed"); let size_res = timeline - .spawn_ondemand_logical_size_calculation(lsn) + .spawn_ondemand_logical_size_calculation(lsn, ctx) .await?; Ok(TimelineAtLsnSizeResult(timeline, lsn, size_res)) } diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index b7ad8fe791..6cb19786e5 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -65,6 +65,7 @@ async fn compaction_loop(tenant_id: TenantId) { ControlFlow::Continue(tenant) => tenant, }, }; + let tenant_ctx = tenant.get_context(); let mut sleep_duration = tenant.get_compaction_period(); if sleep_duration == Duration::ZERO { @@ -73,7 +74,7 @@ async fn compaction_loop(tenant_id: TenantId) { sleep_duration = Duration::from_secs(10); } else { // Run compaction - if let Err(e) = tenant.compaction_iteration().await { + if let Err(e) = tenant.compaction_iteration(&tenant_ctx).await { sleep_duration = wait_duration; error!("Compaction failed, retrying in {:?}: {e:?}", sleep_duration); } @@ -116,6 +117,7 @@ async fn gc_loop(tenant_id: TenantId) { ControlFlow::Continue(tenant) => tenant, }, }; + let tenant_ctx = tenant.get_context(); let gc_period = tenant.get_gc_period(); let gc_horizon = tenant.get_gc_horizon(); @@ -127,7 +129,9 @@ async fn gc_loop(tenant_id: TenantId) { } else { // Run gc if gc_horizon > 0 { - if let Err(e) = tenant.gc_iteration(None, gc_horizon, tenant.get_pitr_interval()).await + if let Err(e) = tenant + .gc_iteration(None, gc_horizon, tenant.get_pitr_interval(), &tenant_ctx) + .await { sleep_duration = wait_duration; error!("Gc failed, retrying in {:?}: {e:?}", sleep_duration); diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 4c76ac614d..2e8e01d137 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -23,6 +23,7 @@ use std::sync::atomic::{AtomicI64, Ordering as AtomicOrdering}; use std::sync::{Arc, Mutex, MutexGuard, RwLock, Weak}; use std::time::{Duration, Instant, SystemTime}; +use crate::context::RequestContext; use crate::tenant::remote_timeline_client::{self, index::LayerFileMetadata}; use crate::tenant::storage_layer::{ DeltaFileName, DeltaLayerWriter, ImageFileName, ImageLayerWriter, InMemoryLayer, LayerFileName, @@ -34,6 +35,7 @@ use crate::tenant::{ metadata::{save_metadata, TimelineMetadata}, par_fsync, storage_layer::{PersistentLayer, ValueReconstructResult, ValueReconstructState}, + TenantRequestContext, }; use crate::config::PageServerConf; @@ -410,6 +412,18 @@ impl Timeline { self.latest_gc_cutoff_lsn.read() } + /// Return a new TenantRequestContext. + /// + /// XXX: This is a placeholder. In the next commit, this will + /// check that the timeline is still active. + pub fn get_context(&self) -> TimelineRequestContext { + TimelineRequestContext { + ctx: TenantRequestContext { + ctx: RequestContext {}, + }, + } + } + /// Look up given page version. /// /// If a remote layer file is needed, it is downloaded as part of this @@ -422,7 +436,12 @@ impl Timeline { /// an ancestor branch, for example, or waste a lot of cycles chasing the /// non-existing key. /// - pub async fn get(&self, key: Key, lsn: Lsn) -> Result { + pub async fn get( + &self, + key: Key, + lsn: Lsn, + ctx: &TimelineRequestContext, + ) -> Result { if !lsn.is_valid() { return Err(PageReconstructError::Other(anyhow::anyhow!("Invalid LSN"))); } @@ -450,7 +469,7 @@ impl Timeline { img: cached_page_img, }; - self.get_reconstruct_data(key, lsn, &mut reconstruct_state) + self.get_reconstruct_data(key, lsn, &mut reconstruct_state, ctx) .await?; self.metrics @@ -513,7 +532,7 @@ impl Timeline { /// You should call this before any of the other get_* or list_* functions. Calling /// those functions with an LSN that has been processed yet is an error. /// - pub async fn wait_lsn(&self, lsn: Lsn) -> anyhow::Result<()> { + pub async fn wait_lsn(&self, lsn: Lsn, ctx: &TimelineRequestContext) -> anyhow::Result<()> { anyhow::ensure!(self.is_active(), "Cannot wait for Lsn on inactive timeline"); // This should never be called from the WAL receiver, because that could lead @@ -558,7 +577,7 @@ impl Timeline { self.flush_frozen_layers_and_wait().await } - pub async fn compact(&self) -> anyhow::Result<()> { + pub async fn compact(&self, ctx: &TimelineRequestContext) -> anyhow::Result<()> { let last_record_lsn = self.get_last_record_lsn(); // Last record Lsn could be zero in case the timeline was just created @@ -616,14 +635,16 @@ impl Timeline { .repartition( self.get_last_record_lsn(), self.get_compaction_target_size(), + ctx, ) .await { Ok((partitioning, lsn)) => { // 2. Create new image layers for partitions that have been modified // "enough". - let layer_paths_to_upload = - self.create_image_layers(&partitioning, lsn, false).await?; + let layer_paths_to_upload = self + .create_image_layers(&partitioning, lsn, false, ctx) + .await?; if let Some(remote_client) = &self.remote_client { for (path, layer_metadata) in layer_paths_to_upload { remote_client.schedule_layer_file_upload(&path, &layer_metadata)?; @@ -632,7 +653,7 @@ impl Timeline { // 3. Compact let timer = self.metrics.compact_time_histo.start_timer(); - self.compact_level0(target_file_size).await?; + self.compact_level0(target_file_size, ctx).await?; timer.stop_and_record(); // If `create_image_layers' or `compact_level0` scheduled any @@ -673,7 +694,10 @@ impl Timeline { /// the initial size calculation has not been run (gets triggered on the first size access). /// /// return size and boolean flag that shows if the size is exact - pub fn get_current_logical_size(self: &Arc) -> anyhow::Result<(u64, bool)> { + pub fn get_current_logical_size( + self: &Arc, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<(u64, bool)> { let current_size = self.current_logical_size.current_size()?; debug!("Current size: {current_size:?}"); @@ -683,7 +707,7 @@ impl Timeline { (current_size, self.current_logical_size.initial_part_end) { is_exact = false; - self.try_spawn_size_init_task(init_lsn); + self.try_spawn_size_init_task(init_lsn, ctx); } Ok((size, is_exact)) @@ -887,7 +911,7 @@ impl Timeline { }) } - pub(super) fn maybe_spawn_flush_loop(self: &Arc) { + pub(super) fn maybe_spawn_flush_loop(self: &Arc, ctx: &TimelineRequestContext) { let mut flush_loop_state = self.flush_loop_state.lock().unwrap(); match *flush_loop_state { FlushLoopState::NotStarted => (), @@ -929,7 +953,7 @@ impl Timeline { *flush_loop_state = FlushLoopState::Running; } - pub(super) fn launch_wal_receiver(self: &Arc) { + pub(super) fn launch_wal_receiver(self: &Arc, ctx: &TimelineRequestContext) { if !is_broker_client_initialized() { if cfg!(test) { info!("not launching WAL receiver because broker client hasn't been initialized"); @@ -939,6 +963,8 @@ impl Timeline { } } + let background_ctx = self.get_context(); + info!( "launching WAL receiver for timeline {} of tenant {}", self.timeline_id, self.tenant_id @@ -961,6 +987,7 @@ impl Timeline { lagging_wal_timeout, max_lsn_wal_lag, crate::config::SAFEKEEPER_AUTH_TOKEN.get().cloned(), + background_ctx, ); } @@ -1220,11 +1247,12 @@ impl Timeline { /// # TODO /// May be a bit cleaner to do things based on populated remote client, /// and then do things based on its upload_queue.latest_files. - #[instrument(skip(self, index_part, up_to_date_metadata))] + #[instrument(skip(self, index_part, up_to_date_metadata, ctx))] pub async fn reconcile_with_remote( &self, up_to_date_metadata: &TimelineMetadata, index_part: Option<&IndexPart>, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { info!("starting"); let remote_client = self @@ -1280,7 +1308,7 @@ impl Timeline { Ok(()) } - fn try_spawn_size_init_task(self: &Arc, init_lsn: Lsn) { + fn try_spawn_size_init_task(self: &Arc, init_lsn: Lsn, ctx: &TimelineRequestContext) { let permit = match Arc::clone(&self.current_logical_size.initial_size_computation) .try_acquire_owned() { @@ -1342,6 +1370,7 @@ impl Timeline { pub fn spawn_ondemand_logical_size_calculation( self: &Arc, lsn: Lsn, + ctx: &TimelineRequestContext, ) -> oneshot::Receiver> { let (sender, receiver) = oneshot::channel(); let self_clone = Arc::clone(self); @@ -1377,7 +1406,8 @@ impl Timeline { // synchronous file IO without .await inbetween // if there are no RemoteLayers that would require downloading. let h = tokio::runtime::Handle::current(); - h.block_on(self_calculation.calculate_logical_size(init_lsn, cancel)) + let ctx = self_calculation.get_context(); + h.block_on(self_calculation.calculate_logical_size(init_lsn, cancel, &ctx)) }) .await .context("Failed to spawn calculation result task")? @@ -1436,6 +1466,7 @@ impl Timeline { &self, up_to_lsn: Lsn, cancel: CancellationToken, + ctx: &TimelineRequestContext, ) -> Result { info!( "Calculating logical size for timeline {} at {}", @@ -1478,7 +1509,7 @@ impl Timeline { self.metrics.logical_size_histo.start_timer() }; let logical_size = self - .get_current_logical_size_non_incremental(up_to_lsn, cancel) + .get_current_logical_size_non_incremental(up_to_lsn, cancel, ctx) .await?; debug!("calculated logical size: {logical_size}"); timer.stop_and_record(); @@ -1555,6 +1586,7 @@ impl Timeline { key: Key, request_lsn: Lsn, reconstruct_state: &mut ValueReconstructState, + ctx: &TimelineRequestContext, ) -> Result<(), PageReconstructError> { // Start from the current timeline. let mut timeline_owned; @@ -1873,6 +1905,7 @@ impl Timeline { /// Layer flusher task's main loop. async fn flush_loop(&self, mut layer_flush_start_rx: tokio::sync::watch::Receiver) { info!("started flush loop"); + let ctx = self.get_context(); loop { tokio::select! { _ = task_mgr::shutdown_watcher() => { @@ -1892,7 +1925,7 @@ impl Timeline { // drop 'layers' lock to allow concurrent reads and writes }; if let Some(layer_to_flush) = layer_to_flush { - if let Err(err) = self.flush_frozen_layer(layer_to_flush).await { + if let Err(err) = self.flush_frozen_layer(layer_to_flush, &ctx).await { error!("could not flush frozen layer: {err:?}"); break Err(err); } @@ -1957,8 +1990,12 @@ impl Timeline { } /// Flush one frozen in-memory layer to disk, as a new delta layer. - #[instrument(skip(self, frozen_layer), fields(tenant_id=%self.tenant_id, timeline_id=%self.timeline_id, layer=%frozen_layer.short_id()))] - async fn flush_frozen_layer(&self, frozen_layer: Arc) -> anyhow::Result<()> { + #[instrument(skip(self, frozen_layer, ctx), fields(tenant_id=%self.tenant_id, timeline_id=%self.timeline_id, layer=%frozen_layer.short_id()))] + async fn flush_frozen_layer( + &self, + frozen_layer: Arc, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<()> { // As a special case, when we have just imported an image into the repository, // instead of writing out a L0 delta layer, we directly write out image layer // files instead. This is possible as long as *all* the data imported into the @@ -1967,9 +2004,9 @@ impl Timeline { let layer_paths_to_upload = if lsn_range.start == self.initdb_lsn && lsn_range.end == Lsn(self.initdb_lsn.0 + 1) { let (partitioning, _lsn) = self - .repartition(self.initdb_lsn, self.get_compaction_target_size()) + .repartition(self.initdb_lsn, self.get_compaction_target_size(), ctx) .await?; - self.create_image_layers(&partitioning, self.initdb_lsn, true) + self.create_image_layers(&partitioning, self.initdb_lsn, true, ctx) .await? } else { // normal case, write out a L0 delta layer file. @@ -2119,6 +2156,7 @@ impl Timeline { &self, lsn: Lsn, partition_size: u64, + ctx: &TimelineRequestContext, ) -> anyhow::Result<(KeyPartitioning, Lsn)> { { let partitioning_guard = self.partitioning.lock().unwrap(); @@ -2129,7 +2167,7 @@ impl Timeline { return Ok((partitioning_guard.0.clone(), partitioning_guard.1)); } } - let keyspace = self.collect_keyspace(lsn).await?; + let keyspace = self.collect_keyspace(lsn, ctx).await?; let partitioning = keyspace.partition(partition_size); let mut partitioning_guard = self.partitioning.lock().unwrap(); @@ -2187,6 +2225,7 @@ impl Timeline { partitioning: &KeyPartitioning, lsn: Lsn, force: bool, + ctx: &TimelineRequestContext, ) -> Result, PageReconstructError> { let timer = self.metrics.create_images_time_histo.start_timer(); let mut image_layers: Vec = Vec::new(); @@ -2211,7 +2250,7 @@ impl Timeline { for range in &partition.ranges { let mut key = range.start; while key < range.end { - let img = match self.get(key, lsn).await { + let img = match self.get(key, lsn, ctx).await { Ok(img) => img, Err(err) => { // If we fail to reconstruct a VM or FSM page, we can zero the @@ -2554,7 +2593,11 @@ impl Timeline { /// Collect a bunch of Level 0 layer files, and compact and reshuffle them as /// as Level 1 files. /// - async fn compact_level0(&self, target_file_size: u64) -> anyhow::Result<()> { + async fn compact_level0( + &self, + target_file_size: u64, + ctx: &TimelineRequestContext, + ) -> anyhow::Result<()> { let CompactLevel0Phase1Result { new_layers, deltas_to_compact, @@ -2570,10 +2613,7 @@ impl Timeline { // Do it here because we don't want to hold self.layers.write() while waiting. if let Some(remote_client) = &self.remote_client { debug!("waiting for upload ops to complete"); - remote_client - .wait_completion() - .await - .context("wait for layer upload ops to complete")?; + remote_client.wait_completion().await?; } let mut layers = self.layers.write().unwrap(); @@ -2662,6 +2702,7 @@ impl Timeline { retain_lsns: Vec, cutoff_horizon: Lsn, pitr: Duration, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // First, calculate pitr_cutoff_timestamp and then convert it to LSN. // @@ -2674,7 +2715,7 @@ impl Timeline { if let Some(pitr_cutoff_timestamp) = now.checked_sub(pitr) { let pitr_timestamp = to_pg_timestamp(pitr_cutoff_timestamp); - match self.find_lsn_for_timestamp(pitr_timestamp).await? { + match self.find_lsn_for_timestamp(pitr_timestamp, ctx).await? { LsnForTimestamp::Present(lsn) => lsn, LsnForTimestamp::Future(lsn) => { // The timestamp is in the future. That sounds impossible, @@ -2724,7 +2765,7 @@ impl Timeline { /// within a layer file. We can only remove the whole file if it's fully /// obsolete. /// - pub(super) async fn gc(&self) -> anyhow::Result { + pub(super) async fn gc(&self, ctx: &TimelineRequestContext) -> anyhow::Result { fail_point!("before-timeline-gc"); let _layer_removal_cs = self.layer_removal_cs.lock().await; @@ -2745,7 +2786,7 @@ impl Timeline { let new_gc_cutoff = Lsn::min(horizon_cutoff, pitr_cutoff); - self.gc_timeline(horizon_cutoff, pitr_cutoff, retain_lsns, new_gc_cutoff) + self.gc_timeline(horizon_cutoff, pitr_cutoff, retain_lsns, new_gc_cutoff, ctx) .instrument( info_span!("gc_timeline", timeline = %self.timeline_id, cutoff = %new_gc_cutoff), ) @@ -2758,6 +2799,7 @@ impl Timeline { pitr_cutoff: Lsn, retain_lsns: Vec, new_gc_cutoff: Lsn, + ctx: &TimelineRequestContext, ) -> anyhow::Result { let now = SystemTime::now(); let mut result: GcResult = GcResult::default(); @@ -2796,10 +2838,7 @@ impl Timeline { // Do it here because we don't want to hold self.layers.write() while waiting. if let Some(remote_client) = &self.remote_client { debug!("waiting for upload ops to complete"); - remote_client - .wait_completion() - .await - .context("wait for layer upload ops to complete")?; + remote_client.wait_completion().await?; } let mut layers_to_remove = Vec::new(); @@ -3116,6 +3155,7 @@ impl Timeline { pub async fn spawn_download_all_remote_layers( self: Arc, + ctx: &TimelineRequestContext, ) -> Result { let mut status_guard = self.download_all_remote_layers_task_info.write().unwrap(); if let Some(st) = &*status_guard { @@ -3130,6 +3170,8 @@ impl Timeline { } } + let child_ctx = self.get_context(); + let self_clone = Arc::clone(&self); let task_id = task_mgr::spawn( task_mgr::BACKGROUND_RUNTIME.handle(), @@ -3139,7 +3181,7 @@ impl Timeline { "download all remote layers task", false, async move { - self_clone.download_all_remote_layers().await; + self_clone.download_all_remote_layers(&child_ctx).await; let mut status_guard = self_clone.download_all_remote_layers_task_info.write().unwrap(); match &mut *status_guard { None => { @@ -3171,7 +3213,7 @@ impl Timeline { Ok(initial_info) } - async fn download_all_remote_layers(self: &Arc) { + async fn download_all_remote_layers(self: &Arc, ctx: &TimelineRequestContext) { let mut downloads: FuturesUnordered<_> = { let layers = self.layers.read().unwrap(); layers @@ -3335,3 +3377,16 @@ fn rename_to_backup(path: &Path) -> anyhow::Result<()> { bail!("couldn't find an unused backup number for {:?}", path) } + +/// XXX: Placeholder +pub struct TimelineRequestContext { + ctx: TenantRequestContext, +} + +impl Deref for TimelineRequestContext { + type Target = RequestContext; + + fn deref(&self) -> &RequestContext { + &self.ctx.ctx + } +} diff --git a/pageserver/src/walingest.rs b/pageserver/src/walingest.rs index 0de2e6654d..ef96208b95 100644 --- a/pageserver/src/walingest.rs +++ b/pageserver/src/walingest.rs @@ -31,7 +31,7 @@ use tracing::*; use crate::pgdatadir_mapping::*; use crate::tenant::PageReconstructError; -use crate::tenant::Timeline; +use crate::tenant::{Timeline, TimelineRequestContext}; use crate::walrecord::*; use crate::ZERO_PAGE; use pageserver_api::reltag::{RelTag, SlruKind}; @@ -52,10 +52,14 @@ pub struct WalIngest<'a> { } impl<'a> WalIngest<'a> { - pub async fn new(timeline: &Timeline, startpoint: Lsn) -> anyhow::Result { + pub async fn new( + timeline: &'a Timeline, + startpoint: Lsn, + ctx: &'_ TimelineRequestContext, + ) -> anyhow::Result> { // Fetch the latest checkpoint into memory, so that we can compare with it // quickly in `ingest_record` and update it when it changes. - let checkpoint_bytes = timeline.get_checkpoint(startpoint).await?; + let checkpoint_bytes = timeline.get_checkpoint(startpoint, ctx).await?; let checkpoint = CheckPoint::decode(&checkpoint_bytes)?; trace!("CheckPoint.nextXid = {}", checkpoint.nextXid.value); @@ -80,6 +84,7 @@ impl<'a> WalIngest<'a> { lsn: Lsn, modification: &mut DatadirModification<'_>, decoded: &mut DecodedWALRecord, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { modification.lsn = lsn; decode_wal_record(recdata, decoded, self.timeline.pg_version)?; @@ -97,7 +102,7 @@ impl<'a> WalIngest<'a> { if decoded.xl_rmid == pg_constants::RM_HEAP_ID || decoded.xl_rmid == pg_constants::RM_HEAP2_ID { - self.ingest_heapam_record(&mut buf, modification, decoded) + self.ingest_heapam_record(&mut buf, modification, decoded, ctx) .await?; } // Handle other special record types @@ -106,13 +111,14 @@ impl<'a> WalIngest<'a> { == pg_constants::XLOG_SMGR_CREATE { let create = XlSmgrCreate::decode(&mut buf); - self.ingest_xlog_smgr_create(modification, &create).await?; + self.ingest_xlog_smgr_create(modification, &create, ctx) + .await?; } else if decoded.xl_rmid == pg_constants::RM_SMGR_ID && (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) == pg_constants::XLOG_SMGR_TRUNCATE { let truncate = XlSmgrTruncate::decode(&mut buf); - self.ingest_xlog_smgr_truncate(modification, &truncate) + self.ingest_xlog_smgr_truncate(modification, &truncate, ctx) .await?; } else if decoded.xl_rmid == pg_constants::RM_DBASE_ID { debug!( @@ -126,7 +132,7 @@ impl<'a> WalIngest<'a> { let createdb = XlCreateDatabase::decode(&mut buf); debug!("XLOG_DBASE_CREATE v14"); - self.ingest_xlog_dbase_create(modification, &createdb) + self.ingest_xlog_dbase_create(modification, &createdb, ctx) .await?; } else if (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) == postgres_ffi::v14::bindings::XLOG_DBASE_DROP @@ -134,7 +140,9 @@ impl<'a> WalIngest<'a> { let dropdb = XlDropDatabase::decode(&mut buf); for tablespace_id in dropdb.tablespace_ids { trace!("Drop db {}, {}", tablespace_id, dropdb.db_id); - modification.drop_dbdir(tablespace_id, dropdb.db_id).await?; + modification + .drop_dbdir(tablespace_id, dropdb.db_id, ctx) + .await?; } } } else if self.timeline.pg_version == 15 { @@ -150,7 +158,7 @@ impl<'a> WalIngest<'a> { // So we can reuse XlCreateDatabase here. debug!("XLOG_DBASE_CREATE_FILE_COPY"); let createdb = XlCreateDatabase::decode(&mut buf); - self.ingest_xlog_dbase_create(modification, &createdb) + self.ingest_xlog_dbase_create(modification, &createdb, ctx) .await?; } else if (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK) == postgres_ffi::v15::bindings::XLOG_DBASE_DROP @@ -158,7 +166,9 @@ impl<'a> WalIngest<'a> { let dropdb = XlDropDatabase::decode(&mut buf); for tablespace_id in dropdb.tablespace_ids { trace!("Drop db {}, {}", tablespace_id, dropdb.db_id); - modification.drop_dbdir(tablespace_id, dropdb.db_id).await?; + modification + .drop_dbdir(tablespace_id, dropdb.db_id, ctx) + .await?; } } } @@ -176,12 +186,13 @@ impl<'a> WalIngest<'a> { segno, rpageno, ZERO_PAGE.clone(), + ctx, ) .await?; } else { assert!(info == pg_constants::CLOG_TRUNCATE); let xlrec = XlClogTruncate::decode(&mut buf); - self.ingest_clog_truncate_record(modification, &xlrec) + self.ingest_clog_truncate_record(modification, &xlrec, ctx) .await?; } } else if decoded.xl_rmid == pg_constants::RM_XACT_ID { @@ -193,6 +204,7 @@ impl<'a> WalIngest<'a> { modification, &parsed_xact, info == pg_constants::XLOG_XACT_COMMIT, + ctx, ) .await?; } else if info == pg_constants::XLOG_XACT_COMMIT_PREPARED @@ -204,6 +216,7 @@ impl<'a> WalIngest<'a> { modification, &parsed_xact, info == pg_constants::XLOG_XACT_COMMIT_PREPARED, + ctx, ) .await?; // Remove twophase file. see RemoveTwoPhaseFile() in postgres code @@ -213,10 +226,12 @@ impl<'a> WalIngest<'a> { parsed_xact.xid, lsn, ); - modification.drop_twophase_file(parsed_xact.xid).await?; + modification + .drop_twophase_file(parsed_xact.xid, ctx) + .await?; } else if info == pg_constants::XLOG_XACT_PREPARE { modification - .put_twophase_file(decoded.xl_xid, Bytes::copy_from_slice(&buf[..])) + .put_twophase_file(decoded.xl_xid, Bytes::copy_from_slice(&buf[..]), ctx) .await?; } } else if decoded.xl_rmid == pg_constants::RM_MULTIXACT_ID { @@ -232,6 +247,7 @@ impl<'a> WalIngest<'a> { segno, rpageno, ZERO_PAGE.clone(), + ctx, ) .await?; } else if info == pg_constants::XLOG_MULTIXACT_ZERO_MEM_PAGE { @@ -244,6 +260,7 @@ impl<'a> WalIngest<'a> { segno, rpageno, ZERO_PAGE.clone(), + ctx, ) .await?; } else if info == pg_constants::XLOG_MULTIXACT_CREATE_ID { @@ -251,12 +268,12 @@ impl<'a> WalIngest<'a> { self.ingest_multixact_create_record(modification, &xlrec)?; } else if info == pg_constants::XLOG_MULTIXACT_TRUNCATE_ID { let xlrec = XlMultiXactTruncate::decode(&mut buf); - self.ingest_multixact_truncate_record(modification, &xlrec) + self.ingest_multixact_truncate_record(modification, &xlrec, ctx) .await?; } } else if decoded.xl_rmid == pg_constants::RM_RELMAP_ID { let xlrec = XlRelmapUpdate::decode(&mut buf); - self.ingest_relmap_page(modification, &xlrec, decoded) + self.ingest_relmap_page(modification, &xlrec, decoded, ctx) .await?; } else if decoded.xl_rmid == pg_constants::RM_XLOG_ID { let info = decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK; @@ -292,7 +309,7 @@ impl<'a> WalIngest<'a> { // Iterate through all the blocks that the record modifies, and // "put" a separate copy of the record for each block. for blk in decoded.blocks.iter() { - self.ingest_decoded_block(modification, lsn, decoded, blk) + self.ingest_decoded_block(modification, lsn, decoded, blk, ctx) .await?; } @@ -317,6 +334,7 @@ impl<'a> WalIngest<'a> { lsn: Lsn, decoded: &DecodedWALRecord, blk: &DecodedBkpBlock, + ctx: &TimelineRequestContext, ) -> Result<(), PageReconstructError> { let rel = RelTag { spcnode: blk.rnode_spcnode, @@ -359,14 +377,14 @@ impl<'a> WalIngest<'a> { page_set_lsn(&mut image, lsn) } assert_eq!(image.len(), BLCKSZ as usize); - self.put_rel_page_image(modification, rel, blk.blkno, image.freeze()) + self.put_rel_page_image(modification, rel, blk.blkno, image.freeze(), ctx) .await?; } else { let rec = NeonWalRecord::Postgres { will_init: blk.will_init || blk.apply_image, rec: decoded.record.clone(), }; - self.put_rel_wal_record(modification, rel, blk.blkno, rec) + self.put_rel_wal_record(modification, rel, blk.blkno, rec, ctx) .await?; } Ok(()) @@ -377,6 +395,7 @@ impl<'a> WalIngest<'a> { buf: &mut Bytes, modification: &mut DatadirModification<'_>, decoded: &mut DecodedWALRecord, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // Handle VM bit updates that are implicitly part of heap records. @@ -456,7 +475,7 @@ impl<'a> WalIngest<'a> { // replaying it would fail to find the previous image of the page, because // it doesn't exist. So check if the VM page(s) exist, and skip the WAL // record if it doesn't. - let vm_size = self.get_relsize(vm_rel, modification.lsn).await?; + let vm_size = self.get_relsize(vm_rel, modification.lsn, ctx).await?; if let Some(blknum) = new_vm_blk { if blknum >= vm_size { new_vm_blk = None; @@ -481,6 +500,7 @@ impl<'a> WalIngest<'a> { old_heap_blkno, flags: pg_constants::VISIBILITYMAP_VALID_BITS, }, + ctx, ) .await?; } else { @@ -496,6 +516,7 @@ impl<'a> WalIngest<'a> { old_heap_blkno: None, flags: pg_constants::VISIBILITYMAP_VALID_BITS, }, + ctx, ) .await?; } @@ -509,6 +530,7 @@ impl<'a> WalIngest<'a> { old_heap_blkno, flags: pg_constants::VISIBILITYMAP_VALID_BITS, }, + ctx, ) .await?; } @@ -524,6 +546,7 @@ impl<'a> WalIngest<'a> { &mut self, modification: &mut DatadirModification<'_>, rec: &XlCreateDatabase, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { let db_id = rec.db_id; let tablespace_id = rec.tablespace_id; @@ -539,7 +562,7 @@ impl<'a> WalIngest<'a> { let rels = modification .tline - .list_rels(src_tablespace_id, src_db_id, req_lsn) + .list_rels(src_tablespace_id, src_db_id, req_lsn, ctx) .await?; debug!("ingest_xlog_dbase_create: {} rels", rels.len()); @@ -547,10 +570,10 @@ impl<'a> WalIngest<'a> { // Copy relfilemap let filemap = modification .tline - .get_relmap_file(src_tablespace_id, src_db_id, req_lsn) + .get_relmap_file(src_tablespace_id, src_db_id, req_lsn, ctx) .await?; modification - .put_relmap_file(tablespace_id, db_id, filemap) + .put_relmap_file(tablespace_id, db_id, filemap, ctx) .await?; let mut num_rels_copied = 0; @@ -561,7 +584,7 @@ impl<'a> WalIngest<'a> { let nblocks = modification .tline - .get_rel_size(src_rel, req_lsn, true) + .get_rel_size(src_rel, req_lsn, true, ctx) .await?; let dst_rel = RelTag { spcnode: tablespace_id, @@ -570,7 +593,7 @@ impl<'a> WalIngest<'a> { forknum: src_rel.forknum, }; - modification.put_rel_creation(dst_rel, nblocks).await?; + modification.put_rel_creation(dst_rel, nblocks, ctx).await?; // Copy content debug!("copying rel {} to {}, {} blocks", src_rel, dst_rel, nblocks); @@ -579,7 +602,7 @@ impl<'a> WalIngest<'a> { let content = modification .tline - .get_rel_page_at_lsn(src_rel, blknum, req_lsn, true) + .get_rel_page_at_lsn(src_rel, blknum, req_lsn, true, ctx) .await?; modification.put_rel_page_image(dst_rel, blknum, content)?; num_blocks_copied += 1; @@ -599,6 +622,7 @@ impl<'a> WalIngest<'a> { &mut self, modification: &mut DatadirModification<'_>, rec: &XlSmgrCreate, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { let rel = RelTag { spcnode: rec.rnode.spcnode, @@ -606,7 +630,7 @@ impl<'a> WalIngest<'a> { relnode: rec.rnode.relnode, forknum: rec.forknum, }; - self.put_rel_creation(modification, rel).await?; + self.put_rel_creation(modification, rel, ctx).await?; Ok(()) } @@ -617,6 +641,7 @@ impl<'a> WalIngest<'a> { &mut self, modification: &mut DatadirModification<'_>, rec: &XlSmgrTruncate, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { let spcnode = rec.rnode.spcnode; let dbnode = rec.rnode.dbnode; @@ -629,7 +654,7 @@ impl<'a> WalIngest<'a> { relnode, forknum: MAIN_FORKNUM, }; - self.put_rel_truncation(modification, rel, rec.blkno) + self.put_rel_truncation(modification, rel, rec.blkno, ctx) .await?; } if (rec.flags & pg_constants::SMGR_TRUNCATE_FSM) != 0 { @@ -648,10 +673,10 @@ impl<'a> WalIngest<'a> { modification.put_rel_page_image(rel, fsm_physical_page_no, ZERO_PAGE.clone())?; fsm_physical_page_no += 1; } - let nblocks = self.get_relsize(rel, modification.lsn).await?; + let nblocks = self.get_relsize(rel, modification.lsn, ctx).await?; if nblocks > fsm_physical_page_no { // check if something to do: FSM is larger than truncate position - self.put_rel_truncation(modification, rel, fsm_physical_page_no) + self.put_rel_truncation(modification, rel, fsm_physical_page_no, ctx) .await?; } } @@ -670,10 +695,10 @@ impl<'a> WalIngest<'a> { modification.put_rel_page_image(rel, vm_page_no, ZERO_PAGE.clone())?; vm_page_no += 1; } - let nblocks = self.get_relsize(rel, modification.lsn).await?; + let nblocks = self.get_relsize(rel, modification.lsn, ctx).await?; if nblocks > vm_page_no { // check if something to do: VM is larger than truncate position - self.put_rel_truncation(modification, rel, vm_page_no) + self.put_rel_truncation(modification, rel, vm_page_no, ctx) .await?; } } @@ -687,6 +712,7 @@ impl<'a> WalIngest<'a> { modification: &mut DatadirModification<'_>, parsed: &XlXactParsedRecord, is_commit: bool, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // Record update of CLOG pages let mut pageno = parsed.xid / pg_constants::CLOG_XACTS_PER_PAGE; @@ -745,10 +771,10 @@ impl<'a> WalIngest<'a> { let last_lsn = self.timeline.get_last_record_lsn(); if modification .tline - .get_rel_exists(rel, last_lsn, true) + .get_rel_exists(rel, last_lsn, true, ctx) .await? { - self.put_rel_drop(modification, rel).await?; + self.put_rel_drop(modification, rel, ctx).await?; } } } @@ -759,6 +785,7 @@ impl<'a> WalIngest<'a> { &mut self, modification: &mut DatadirModification<'_>, xlrec: &XlClogTruncate, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { info!( "RM_CLOG_ID truncate pageno {} oldestXid {} oldestXidDB {}", @@ -799,16 +826,15 @@ impl<'a> WalIngest<'a> { // it. So we use the previous record's LSN in the get calls // instead. let req_lsn = modification.tline.get_last_record_lsn(); - - let slru_segments = modification + for segno in modification .tline - .list_slru_segments(SlruKind::Clog, req_lsn) - .await?; - for segno in slru_segments { + .list_slru_segments(SlruKind::Clog, req_lsn, ctx) + .await? + { let segpage = segno * pg_constants::SLRU_PAGES_PER_SEGMENT; if slru_may_delete_clogsegment(segpage, xlrec.pageno) { modification - .drop_slru_segment(SlruKind::Clog, segno) + .drop_slru_segment(SlruKind::Clog, segno, ctx) .await?; trace!("Drop CLOG segment {:>04X}", segno); } @@ -900,6 +926,7 @@ impl<'a> WalIngest<'a> { &mut self, modification: &mut DatadirModification<'_>, xlrec: &XlMultiXactTruncate, + ctx: &TimelineRequestContext, ) -> Result<()> { self.checkpoint.oldestMulti = xlrec.end_trunc_off; self.checkpoint.oldestMultiDB = xlrec.oldest_multi_db; @@ -915,7 +942,7 @@ impl<'a> WalIngest<'a> { // contain, possibly partially, valid data. while segment != endsegment { modification - .drop_slru_segment(SlruKind::MultiXactMembers, segment as u32) + .drop_slru_segment(SlruKind::MultiXactMembers, segment as u32, ctx) .await?; /* move to next segment, handling wraparound correctly */ @@ -937,6 +964,7 @@ impl<'a> WalIngest<'a> { modification: &mut DatadirModification<'_>, xlrec: &XlRelmapUpdate, decoded: &DecodedWALRecord, + ctx: &TimelineRequestContext, ) -> Result<()> { let mut buf = decoded.record.clone(); buf.advance(decoded.main_data_offset); @@ -944,18 +972,22 @@ impl<'a> WalIngest<'a> { buf.advance(12); modification - .put_relmap_file(xlrec.tsid, xlrec.dbid, Bytes::copy_from_slice(&buf[..])) - .await?; - - Ok(()) + .put_relmap_file( + xlrec.tsid, + xlrec.dbid, + Bytes::copy_from_slice(&buf[..]), + ctx, + ) + .await } async fn put_rel_creation( &mut self, modification: &mut DatadirModification<'_>, rel: RelTag, + ctx: &TimelineRequestContext, ) -> Result<()> { - modification.put_rel_creation(rel, 0).await?; + modification.put_rel_creation(rel, 0, ctx).await?; Ok(()) } @@ -965,8 +997,10 @@ impl<'a> WalIngest<'a> { rel: RelTag, blknum: BlockNumber, img: Bytes, - ) -> anyhow::Result<()> { - self.handle_rel_extend(modification, rel, blknum).await?; + ctx: &TimelineRequestContext, + ) -> Result<(), PageReconstructError> { + self.handle_rel_extend(modification, rel, blknum, ctx) + .await?; modification.put_rel_page_image(rel, blknum, img)?; Ok(()) } @@ -977,8 +1011,10 @@ impl<'a> WalIngest<'a> { rel: RelTag, blknum: BlockNumber, rec: NeonWalRecord, - ) -> anyhow::Result<()> { - self.handle_rel_extend(modification, rel, blknum).await?; + ctx: &TimelineRequestContext, + ) -> Result<()> { + self.handle_rel_extend(modification, rel, blknum, ctx) + .await?; modification.put_rel_wal_record(rel, blknum, rec)?; Ok(()) } @@ -988,8 +1024,9 @@ impl<'a> WalIngest<'a> { modification: &mut DatadirModification<'_>, rel: RelTag, nblocks: BlockNumber, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { - modification.put_rel_truncation(rel, nblocks).await?; + modification.put_rel_truncation(rel, nblocks, ctx).await?; Ok(()) } @@ -997,17 +1034,22 @@ impl<'a> WalIngest<'a> { &mut self, modification: &mut DatadirModification<'_>, rel: RelTag, + ctx: &TimelineRequestContext, ) -> Result<()> { - modification.put_rel_drop(rel).await?; + modification.put_rel_drop(rel, ctx).await?; Ok(()) } - async fn get_relsize(&mut self, rel: RelTag, lsn: Lsn) -> anyhow::Result { - let exists = self.timeline.get_rel_exists(rel, lsn, true).await?; - let nblocks = if !exists { + async fn get_relsize( + &mut self, + rel: RelTag, + lsn: Lsn, + ctx: &TimelineRequestContext, + ) -> anyhow::Result { + let nblocks = if !self.timeline.get_rel_exists(rel, lsn, true, ctx).await? { 0 } else { - self.timeline.get_rel_size(rel, lsn, true).await? + self.timeline.get_rel_size(rel, lsn, true, ctx).await? }; Ok(nblocks) } @@ -1017,23 +1059,28 @@ impl<'a> WalIngest<'a> { modification: &mut DatadirModification<'_>, rel: RelTag, blknum: BlockNumber, - ) -> anyhow::Result<()> { + ctx: &TimelineRequestContext, + ) -> Result<(), PageReconstructError> { let new_nblocks = blknum + 1; // Check if the relation exists. We implicitly create relations on first // record. // TODO: would be nice if to be more explicit about it let last_lsn = modification.lsn; - let old_nblocks = if !self.timeline.get_rel_exists(rel, last_lsn, true).await? { + let old_nblocks = if !self + .timeline + .get_rel_exists(rel, last_lsn, true, ctx) + .await? + { // create it with 0 size initially, the logic below will extend it - modification.put_rel_creation(rel, 0).await?; + modification.put_rel_creation(rel, 0, ctx).await?; 0 } else { - self.timeline.get_rel_size(rel, last_lsn, true).await? + self.timeline.get_rel_size(rel, last_lsn, true, ctx).await? }; if new_nblocks > old_nblocks { //info!("extending {} {} to {}", rel, old_nblocks, new_nblocks); - modification.put_rel_extend(rel, new_nblocks).await?; + modification.put_rel_extend(rel, new_nblocks, ctx).await?; // fill the gap with zeros for gap_blknum in old_nblocks..blknum { @@ -1050,8 +1097,9 @@ impl<'a> WalIngest<'a> { segno: u32, blknum: BlockNumber, img: Bytes, - ) -> anyhow::Result<()> { - self.handle_slru_extend(modification, kind, segno, blknum) + ctx: &TimelineRequestContext, + ) -> Result<()> { + self.handle_slru_extend(modification, kind, segno, blknum, ctx) .await?; modification.put_slru_page_image(kind, segno, blknum, img)?; Ok(()) @@ -1063,6 +1111,7 @@ impl<'a> WalIngest<'a> { kind: SlruKind, segno: u32, blknum: BlockNumber, + ctx: &TimelineRequestContext, ) -> anyhow::Result<()> { // we don't use a cache for this like we do for relations. SLRUS are explcitly // extended with ZEROPAGE records, not with commit records, so it happens @@ -1075,17 +1124,17 @@ impl<'a> WalIngest<'a> { let last_lsn = self.timeline.get_last_record_lsn(); let old_nblocks = if !self .timeline - .get_slru_segment_exists(kind, segno, last_lsn) + .get_slru_segment_exists(kind, segno, last_lsn, ctx) .await? { // create it with 0 size initially, the logic below will extend it modification - .put_slru_segment_creation(kind, segno, 0) + .put_slru_segment_creation(kind, segno, 0, ctx) .await?; 0 } else { self.timeline - .get_slru_segment_size(kind, segno, last_lsn) + .get_slru_segment_size(kind, segno, last_lsn, ctx) .await? }; @@ -1134,12 +1183,15 @@ mod tests { static ZERO_CHECKPOINT: Bytes = Bytes::from_static(&[0u8; SIZEOF_CHECKPOINT]); - async fn init_walingest_test(tline: &Timeline) -> Result { + async fn init_walingest_test<'a>( + tline: &'a Timeline, + ctx: &TimelineRequestContext, + ) -> Result> { let mut m = tline.begin_modification(Lsn(0x10)); m.put_checkpoint(ZERO_CHECKPOINT.clone())?; - m.put_relmap_file(0, 111, Bytes::from("")).await?; // dummy relmapper file + m.put_relmap_file(0, 111, Bytes::from(""), ctx).await?; // dummy relmapper file m.commit()?; - let walingest = WalIngest::new(tline, Lsn(0x10)).await?; + let walingest = WalIngest::new(tline, Lsn(0x10), ctx).await?; Ok(walingest) } @@ -1147,28 +1199,30 @@ mod tests { #[tokio::test] async fn test_relsize() -> Result<()> { let tenant = TenantHarness::create("test_relsize")?.load().await; - let tline = create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION)?; - let mut walingest = init_walingest_test(&tline).await?; + let tenant_ctx = tenant.get_context(); + let (tline, ctx) = + create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION, &tenant_ctx)?; + let mut walingest = init_walingest_test(&tline, &ctx).await?; let mut m = tline.begin_modification(Lsn(0x20)); - walingest.put_rel_creation(&mut m, TESTREL_A).await?; + walingest.put_rel_creation(&mut m, TESTREL_A, &ctx).await?; walingest - .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2")) + .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"), &ctx) .await?; m.commit()?; let mut m = tline.begin_modification(Lsn(0x30)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 3")) + .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 3"), &ctx) .await?; m.commit()?; let mut m = tline.begin_modification(Lsn(0x40)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1 at 4")) + .put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1 at 4"), &ctx) .await?; m.commit()?; let mut m = tline.begin_modification(Lsn(0x50)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 2, TEST_IMG("foo blk 2 at 5")) + .put_rel_page_image(&mut m, TESTREL_A, 2, TEST_IMG("foo blk 2 at 5"), &ctx) .await?; m.commit()?; @@ -1176,120 +1230,157 @@ mod tests { // The relation was created at LSN 2, not visible at LSN 1 yet. assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x10), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x10), false, &ctx) + .await?, false ); assert!(tline - .get_rel_size(TESTREL_A, Lsn(0x10), false) + .get_rel_size(TESTREL_A, Lsn(0x10), false, &ctx) .await .is_err()); - assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x20), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x20), false, &ctx) + .await?, true ); - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20), false).await?, 1); - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50), false).await?, 3); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x20), false, &ctx) + .await?, + 1 + ); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x50), false, &ctx) + .await?, + 3 + ); // Check page contents at each LSN assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x20), false) + .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x20), false, &ctx) .await?, TEST_IMG("foo blk 0 at 2") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x30), false) + .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x30), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x40), false) + .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x40), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40), false) + .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40), false, &ctx) .await?, TEST_IMG("foo blk 1 at 4") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x50), false) + .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x50), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50), false) + .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50), false, &ctx) .await?, TEST_IMG("foo blk 1 at 4") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false) + .get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false, &ctx) .await?, TEST_IMG("foo blk 2 at 5") ); // Truncate last block let mut m = tline.begin_modification(Lsn(0x60)); - walingest.put_rel_truncation(&mut m, TESTREL_A, 2).await?; + walingest + .put_rel_truncation(&mut m, TESTREL_A, 2, &ctx) + .await?; m.commit()?; assert_current_logical_size(&tline, Lsn(0x60)); // Check reported size and contents after truncation - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x60), false).await?, 2); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x60), false) + .get_rel_size(TESTREL_A, Lsn(0x60), false, &ctx) + .await?, + 2 + ); + assert_eq!( + tline + .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x60), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60), false) + .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60), false, &ctx) .await?, TEST_IMG("foo blk 1 at 4") ); // should still see the truncated block with older LSN - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50), false).await?, 3); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false) + .get_rel_size(TESTREL_A, Lsn(0x50), false, &ctx) + .await?, + 3 + ); + assert_eq!( + tline + .get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false, &ctx) .await?, TEST_IMG("foo blk 2 at 5") ); // Truncate to zero length let mut m = tline.begin_modification(Lsn(0x68)); - walingest.put_rel_truncation(&mut m, TESTREL_A, 0).await?; + walingest + .put_rel_truncation(&mut m, TESTREL_A, 0, &ctx) + .await?; m.commit()?; - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x68), false).await?, 0); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x68), false, &ctx) + .await?, + 0 + ); // Extend from 0 to 2 blocks, leaving a gap let mut m = tline.begin_modification(Lsn(0x70)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1")) + .put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1"), &ctx) .await?; m.commit()?; - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x70), false).await?, 2); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x70), false) + .get_rel_size(TESTREL_A, Lsn(0x70), false, &ctx) + .await?, + 2 + ); + assert_eq!( + tline + .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x70), false, &ctx) .await?, ZERO_PAGE ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x70), false) + .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x70), false, &ctx) .await?, TEST_IMG("foo blk 1") ); @@ -1297,21 +1388,26 @@ mod tests { // Extend a lot more, leaving a big gap that spans across segments let mut m = tline.begin_modification(Lsn(0x80)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 1500, TEST_IMG("foo blk 1500")) + .put_rel_page_image(&mut m, TESTREL_A, 1500, TEST_IMG("foo blk 1500"), &ctx) .await?; m.commit()?; - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x80), false).await?, 1501); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x80), false, &ctx) + .await?, + 1501 + ); for blk in 2..1500 { assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blk, Lsn(0x80), false) + .get_rel_page_at_lsn(TESTREL_A, blk, Lsn(0x80), false, &ctx) .await?, ZERO_PAGE ); } assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1500, Lsn(0x80), false) + .get_rel_page_at_lsn(TESTREL_A, 1500, Lsn(0x80), false, &ctx) .await?, TEST_IMG("foo blk 1500") ); @@ -1324,30 +1420,41 @@ mod tests { #[tokio::test] async fn test_drop_extend() -> Result<()> { let tenant = TenantHarness::create("test_drop_extend")?.load().await; - let tline = create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION)?; - let mut walingest = init_walingest_test(&tline).await?; + let tenant_ctx = tenant.get_context(); + let (tline, ctx) = + create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION, &tenant_ctx)?; + let mut walingest = init_walingest_test(&tline, &ctx).await?; let mut m = tline.begin_modification(Lsn(0x20)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2")) + .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"), &ctx) .await?; m.commit()?; // Check that rel exists and size is correct assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x20), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x20), false, &ctx) + .await?, true ); - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20), false).await?, 1); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x20), false, &ctx) + .await?, + 1 + ); // Drop rel let mut m = tline.begin_modification(Lsn(0x30)); - walingest.put_rel_drop(&mut m, TESTREL_A).await?; + walingest.put_rel_drop(&mut m, TESTREL_A, &ctx).await?; m.commit()?; // Check that rel is not visible anymore assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x30), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x30), false, &ctx) + .await?, false ); @@ -1357,16 +1464,23 @@ mod tests { // Re-create it let mut m = tline.begin_modification(Lsn(0x40)); walingest - .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 4")) + .put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 4"), &ctx) .await?; m.commit()?; // Check that rel exists and size is correct assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x40), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x40), false, &ctx) + .await?, true ); - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x40), false).await?, 1); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x40), false, &ctx) + .await?, + 1 + ); Ok(()) } @@ -1377,8 +1491,10 @@ mod tests { #[tokio::test] async fn test_truncate_extend() -> Result<()> { let tenant = TenantHarness::create("test_truncate_extend")?.load().await; - let tline = create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION)?; - let mut walingest = init_walingest_test(&tline).await?; + let tenant_ctx = tenant.get_context(); + let (tline, ctx) = + create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION, &tenant_ctx)?; + let mut walingest = init_walingest_test(&tline, &ctx).await?; // Create a 20 MB relation (the size is arbitrary) let relsize = 20 * 1024 * 1024 / 8192; @@ -1386,27 +1502,33 @@ mod tests { for blkno in 0..relsize { let data = format!("foo blk {} at {}", blkno, Lsn(0x20)); walingest - .put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data)) + .put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data), &ctx) .await?; } m.commit()?; // The relation was created at LSN 20, not visible at LSN 1 yet. assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x10), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x10), false, &ctx) + .await?, false ); assert!(tline - .get_rel_size(TESTREL_A, Lsn(0x10), false) + .get_rel_size(TESTREL_A, Lsn(0x10), false, &ctx) .await .is_err()); assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x20), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x20), false, &ctx) + .await?, true ); assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(0x20), false).await?, + tline + .get_rel_size(TESTREL_A, Lsn(0x20), false, &ctx) + .await?, relsize ); @@ -1416,7 +1538,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, lsn, false) + .get_rel_page_at_lsn(TESTREL_A, blkno, lsn, false, &ctx) .await?, TEST_IMG(&data) ); @@ -1425,18 +1547,25 @@ mod tests { // Truncate relation so that second segment was dropped // - only leave one page let mut m = tline.begin_modification(Lsn(0x60)); - walingest.put_rel_truncation(&mut m, TESTREL_A, 1).await?; + walingest + .put_rel_truncation(&mut m, TESTREL_A, 1, &ctx) + .await?; m.commit()?; // Check reported size and contents after truncation - assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x60), false).await?, 1); + assert_eq!( + tline + .get_rel_size(TESTREL_A, Lsn(0x60), false, &ctx) + .await?, + 1 + ); for blkno in 0..1 { let lsn = Lsn(0x20); let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x60), false) + .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x60), false, &ctx) .await?, TEST_IMG(&data) ); @@ -1444,7 +1573,9 @@ mod tests { // should still see all blocks with older LSN assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(0x50), false).await?, + tline + .get_rel_size(TESTREL_A, Lsn(0x50), false, &ctx) + .await?, relsize ); for blkno in 0..relsize { @@ -1452,7 +1583,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x50), false) + .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x50), false, &ctx) .await?, TEST_IMG(&data) ); @@ -1465,17 +1596,21 @@ mod tests { for blkno in 0..relsize { let data = format!("foo blk {} at {}", blkno, lsn); walingest - .put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data)) + .put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data), &ctx) .await?; } m.commit()?; assert_eq!( - tline.get_rel_exists(TESTREL_A, Lsn(0x80), false).await?, + tline + .get_rel_exists(TESTREL_A, Lsn(0x80), false, &ctx) + .await?, true ); assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(0x80), false).await?, + tline + .get_rel_size(TESTREL_A, Lsn(0x80), false, &ctx) + .await?, relsize ); // Check relation content @@ -1484,7 +1619,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x80), false) + .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x80), false, &ctx) .await?, TEST_IMG(&data) ); @@ -1498,8 +1633,10 @@ mod tests { #[tokio::test] async fn test_large_rel() -> Result<()> { let tenant = TenantHarness::create("test_large_rel")?.load().await; - let tline = create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION)?; - let mut walingest = init_walingest_test(&tline).await?; + let tenant_ctx = tenant.get_context(); + let (tline, ctx) = + create_test_timeline(&tenant, TIMELINE_ID, DEFAULT_PG_VERSION, &tenant_ctx)?; + let mut walingest = init_walingest_test(&tline, &ctx).await?; let mut lsn = 0x10; for blknum in 0..RELSEG_SIZE + 1 { @@ -1507,7 +1644,7 @@ mod tests { let mut m = tline.begin_modification(Lsn(lsn)); let img = TEST_IMG(&format!("foo blk {} at {}", blknum, Lsn(lsn))); walingest - .put_rel_page_image(&mut m, TESTREL_A, blknum as BlockNumber, img) + .put_rel_page_image(&mut m, TESTREL_A, blknum as BlockNumber, img, &ctx) .await?; m.commit()?; } @@ -1515,7 +1652,7 @@ mod tests { assert_current_logical_size(&tline, Lsn(lsn)); assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false).await?, + tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, RELSEG_SIZE + 1 ); @@ -1523,11 +1660,11 @@ mod tests { lsn += 0x10; let mut m = tline.begin_modification(Lsn(lsn)); walingest - .put_rel_truncation(&mut m, TESTREL_A, RELSEG_SIZE) + .put_rel_truncation(&mut m, TESTREL_A, RELSEG_SIZE, &ctx) .await?; m.commit()?; assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false).await?, + tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, RELSEG_SIZE ); assert_current_logical_size(&tline, Lsn(lsn)); @@ -1536,11 +1673,11 @@ mod tests { lsn += 0x10; let mut m = tline.begin_modification(Lsn(lsn)); walingest - .put_rel_truncation(&mut m, TESTREL_A, RELSEG_SIZE - 1) + .put_rel_truncation(&mut m, TESTREL_A, RELSEG_SIZE - 1, &ctx) .await?; m.commit()?; assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false).await?, + tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, RELSEG_SIZE - 1 ); assert_current_logical_size(&tline, Lsn(lsn)); @@ -1552,11 +1689,11 @@ mod tests { lsn += 0x10; let mut m = tline.begin_modification(Lsn(lsn)); walingest - .put_rel_truncation(&mut m, TESTREL_A, size as BlockNumber) + .put_rel_truncation(&mut m, TESTREL_A, size as BlockNumber, &ctx) .await?; m.commit()?; assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false).await?, + tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, size as BlockNumber ); diff --git a/pageserver/src/walreceiver/connection_manager.rs b/pageserver/src/walreceiver/connection_manager.rs index 8b60e59305..f5f164329d 100644 --- a/pageserver/src/walreceiver/connection_manager.rs +++ b/pageserver/src/walreceiver/connection_manager.rs @@ -13,7 +13,7 @@ use std::{collections::HashMap, num::NonZeroU64, ops::ControlFlow, sync::Arc, ti use crate::task_mgr::TaskKind; use crate::task_mgr::WALRECEIVER_RUNTIME; -use crate::tenant::Timeline; +use crate::tenant::{Timeline, TimelineRequestContext}; use crate::{task_mgr, walreceiver::TaskStateUpdate}; use anyhow::Context; use chrono::{NaiveDateTime, Utc}; @@ -46,6 +46,7 @@ pub fn spawn_connection_manager_task( lagging_wal_timeout: Duration, max_lsn_wal_lag: NonZeroU64, auth_token: Option>, + ctx: TimelineRequestContext, ) { let mut broker_client = get_broker_client().clone(); @@ -78,6 +79,7 @@ pub fn spawn_connection_manager_task( loop_step_result = connection_manager_loop_step( &mut broker_client, &mut walreceiver_state, + &ctx, ) => match loop_step_result { ControlFlow::Continue(()) => continue, ControlFlow::Break(()) => { @@ -101,6 +103,7 @@ pub fn spawn_connection_manager_task( async fn connection_manager_loop_step( broker_client: &mut BrokerClientChannel, walreceiver_state: &mut WalreceiverState, + ctx: &TimelineRequestContext, ) -> ControlFlow<(), ()> { let mut timeline_state_updates = walreceiver_state.timeline.subscribe_for_state_updates(); @@ -226,6 +229,7 @@ async fn connection_manager_loop_step( .change_connection( new_candidate.safekeeper_id, new_candidate.wal_source_connconf, + ctx, ) .await } @@ -389,12 +393,14 @@ impl WalreceiverState { &mut self, new_sk_id: NodeId, new_wal_source_connconf: PgConnectionConfig, + ctx: &TimelineRequestContext, ) { self.drop_old_connection(true).await; let id = self.id; let connect_timeout = self.wal_connect_timeout; let timeline = Arc::clone(&self.timeline); + let child_ctx = timeline.get_context(); let connection_handle = TaskHandle::spawn(move |events_sender, cancellation| { async move { super::walreceiver_connection::handle_walreceiver_connection( @@ -403,6 +409,7 @@ impl WalreceiverState { events_sender, cancellation, connect_timeout, + child_ctx, ) .await .context("walreceiver connection handling failure") @@ -1233,18 +1240,18 @@ mod tests { const DUMMY_SAFEKEEPER_HOST: &str = "safekeeper_connstr"; async fn dummy_state(harness: &TenantHarness<'_>) -> WalreceiverState { + let tenant = harness.load().await; + let tenant_ctx = tenant.get_context(); + let (timeline, timeline_ctx) = tenant + .create_empty_timeline(TIMELINE_ID, Lsn(0), crate::DEFAULT_PG_VERSION, &tenant_ctx) + .expect("Failed to create an empty timeline for dummy wal connection manager"); + let timeline = timeline.initialize(&timeline_ctx).unwrap(); WalreceiverState { id: TenantTimelineId { tenant_id: harness.tenant_id, timeline_id: TIMELINE_ID, }, - timeline: harness - .load() - .await - .create_empty_timeline(TIMELINE_ID, Lsn(0), crate::DEFAULT_PG_VERSION) - .expect("Failed to create an empty timeline for dummy wal connection manager") - .initialize() - .unwrap(), + timeline, wal_connect_timeout: Duration::from_secs(1), lagging_wal_timeout: Duration::from_secs(1), max_lsn_wal_lag: NonZeroU64::new(1024 * 1024).unwrap(), diff --git a/pageserver/src/walreceiver/walreceiver_connection.rs b/pageserver/src/walreceiver/walreceiver_connection.rs index 1b9e4923fb..8e64f55ea6 100644 --- a/pageserver/src/walreceiver/walreceiver_connection.rs +++ b/pageserver/src/walreceiver/walreceiver_connection.rs @@ -27,7 +27,7 @@ use crate::{ task_mgr, task_mgr::TaskKind, task_mgr::WALRECEIVER_RUNTIME, - tenant::{Timeline, WalReceiverInfo}, + tenant::{Timeline, TimelineRequestContext, WalReceiverInfo}, walingest::WalIngest, walrecord::DecodedWALRecord, }; @@ -62,6 +62,7 @@ pub async fn handle_walreceiver_connection( events_sender: watch::Sender>, cancellation: CancellationToken, connect_timeout: Duration, + ctx: TimelineRequestContext, ) -> anyhow::Result<()> { // Connect to the database in replication mode. info!("connecting to {wal_source_connconf:?}"); @@ -180,7 +181,7 @@ pub async fn handle_walreceiver_connection( let mut waldecoder = WalStreamDecoder::new(startpoint, timeline.pg_version); - let mut walingest = WalIngest::new(timeline.as_ref(), startpoint).await?; + let mut walingest = WalIngest::new(timeline.as_ref(), startpoint, &ctx).await?; while let Some(replication_message) = { select! { @@ -251,7 +252,7 @@ pub async fn handle_walreceiver_connection( ensure!(lsn.is_aligned()); walingest - .ingest_record(recdata.clone(), lsn, &mut modification, &mut decoded) + .ingest_record(recdata, lsn, &mut modification, &mut decoded, &ctx) .await .with_context(|| format!("could not ingest record at {lsn}"))?; @@ -329,7 +330,7 @@ pub async fn handle_walreceiver_connection( // Send the replication feedback message. // Regular standby_status_update fields are put into this message. let (timeline_logical_size, _) = timeline - .get_current_logical_size() + .get_current_logical_size(&ctx) .context("Status update creation failed to get current logical size")?; let status_update = ReplicationFeedback { current_timeline_size: timeline_logical_size,