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,