mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-05 20:42:54 +00:00
Introduce RequestContext
Motivation ========== Layer Eviction Needs Context ---------------------------- Before we start implementing layer eviction, we need to collect some access statistics per layer file or maybe even page. Part of these statistics should be the initiator of a page read request to answer the question of whether it was page_service vs. one of the background loops, and if the latter, which of them? Further, it would be nice to learn more about what activity in the pageserver initiated an on-demand download of a layer file. We will use this information to test out layer eviction policies. Read more about the current plan for layer eviction here: https://github.com/neondatabase/neon/issues/2476#issuecomment-1370822104 task_mgr problems + cancellation + tenant/timeline lifecycle ------------------------------------------------------------ Apart from layer eviction, we have long-standing problems with task_mgr, task cancellation, and various races around tenant / timeline lifecycle transitions. One approach to solve these is to abandon task_mgr in favor of a mechanism similar to Golang's context.Context, albeit extended to support waiting for completion, and specialized to the needs in the pageserver. Heikki solves all of the above at once in PR https://github.com/neondatabase/neon/pull/3228 , which is not yet merged at the time of writing. What Is This Patch About ======================== This patch addresses the immediate needs of layer eviction by introducing a `RequestContext` structure that is plumbed through the pageserver - all the way from the various entrypoints (page_service, management API, tenant background loops) down to Timeline::{get,get_reconstruct_data}. The struct carries a description of the kind of activity that initiated the call. We re-use task_mgr::TaskKind for this. Also, it carries the desired on-demand download behavior of the entrypoint. Timeline::get_reconstruct_data can then log the TaskKind that initiated the on-demand download. I developed this patch by git-checking-out Heikki's big RequestContext PR https://github.com/neondatabase/neon/pull/3228 , then deleting all the functionality that we do not need to address the needs for layer eviction. After that, I added a few things on top: 1. The concept of attached_child and detached_child in preparation for cancellation signalling through RequestContext, which will be added in a future patch. 2. A kill switch to turn DownloadBehavior::Error into a warning. 3. Renamed WalReceiverConnection to WalReceiverConnectionPoller and added an additional TaskKind WalReceiverConnectionHandler.These were necessary to create proper detached_child-type RequestContexts for the various tasks that walreceiver starts. How To Review This Patch ======================== Start your review with the module-level comment in context.rs. It explains the idea of RequestContext, what parts of it are implemented in this patch, and the future plans for RequestContext. Then review the various `task_mgr::spawn` call sites. At each of them, we should be creating a new detached_child RequestContext. Then review the (few) RequestContext::attached_child call sites and ensure that the spawned tasks do not outlive the task that spawns them. If they do, these call sites should use detached_child() instead. Then review the todo_child() call sites and judge whether it's worth the trouble of plumbing through a parent context from the caller(s). Lastly, go through the bulk of mechanical changes that simply forwards the &ctx.
This commit is contained in:
committed by
Christian Schwarz
parent
dee71404a2
commit
01b4b0c2f3
@@ -27,6 +27,7 @@ use tracing::*;
|
||||
///
|
||||
use tokio_tar::{Builder, EntryType, Header};
|
||||
|
||||
use crate::context::RequestContext;
|
||||
use crate::tenant::Timeline;
|
||||
use pageserver_api::reltag::{RelTag, SlruKind};
|
||||
|
||||
@@ -52,6 +53,7 @@ pub async fn send_basebackup_tarball<'a, W>(
|
||||
req_lsn: Option<Lsn>,
|
||||
prev_lsn: Option<Lsn>,
|
||||
full_backup: bool,
|
||||
ctx: &'a RequestContext,
|
||||
) -> anyhow::Result<()>
|
||||
where
|
||||
W: AsyncWrite + Send + Sync + Unpin,
|
||||
@@ -110,6 +112,7 @@ where
|
||||
lsn: backup_lsn,
|
||||
prev_record_lsn: prev_lsn,
|
||||
full_backup,
|
||||
ctx,
|
||||
};
|
||||
basebackup
|
||||
.send_tarball()
|
||||
@@ -129,6 +132,7 @@ where
|
||||
lsn: Lsn,
|
||||
prev_record_lsn: Lsn,
|
||||
full_backup: bool,
|
||||
ctx: &'a RequestContext,
|
||||
}
|
||||
|
||||
impl<'a, W> Basebackup<'a, W>
|
||||
@@ -171,23 +175,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 +221,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 {
|
||||
@@ -223,7 +244,7 @@ where
|
||||
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 +266,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<u8> = 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 +308,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 +344,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 +377,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 +418,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")?;
|
||||
|
||||
|
||||
@@ -13,6 +13,7 @@ use tracing::*;
|
||||
use metrics::set_build_info_metric;
|
||||
use pageserver::{
|
||||
config::{defaults::*, PageServerConf},
|
||||
context::{DownloadBehavior, RequestContext},
|
||||
http, page_cache, page_service, task_mgr,
|
||||
task_mgr::TaskKind,
|
||||
task_mgr::{
|
||||
@@ -328,6 +329,13 @@ fn start_pageserver(conf: &'static PageServerConf) -> anyhow::Result<()> {
|
||||
);
|
||||
|
||||
if let Some(metric_collection_endpoint) = &conf.metric_collection_endpoint {
|
||||
let metrics_ctx = RequestContext::todo_child(
|
||||
TaskKind::MetricsCollection,
|
||||
// This task itself shouldn't download anything.
|
||||
// The actual size calculation does need downloads, and
|
||||
// creates a child context with the right DownloadBehavior.
|
||||
DownloadBehavior::Error,
|
||||
);
|
||||
task_mgr::spawn(
|
||||
MGMT_REQUEST_RUNTIME.handle(),
|
||||
TaskKind::MetricsCollection,
|
||||
@@ -341,6 +349,7 @@ fn start_pageserver(conf: &'static PageServerConf) -> anyhow::Result<()> {
|
||||
conf.metric_collection_interval,
|
||||
conf.synthetic_size_calculation_interval,
|
||||
conf.id,
|
||||
metrics_ctx,
|
||||
)
|
||||
.instrument(info_span!("metrics_collection"))
|
||||
.await?;
|
||||
@@ -352,17 +361,34 @@ fn start_pageserver(conf: &'static PageServerConf) -> anyhow::Result<()> {
|
||||
|
||||
// Spawn a task to listen for libpq connections. It will spawn further tasks
|
||||
// for each connection. We created the listener earlier already.
|
||||
task_mgr::spawn(
|
||||
COMPUTE_REQUEST_RUNTIME.handle(),
|
||||
TaskKind::LibpqEndpointListener,
|
||||
None,
|
||||
None,
|
||||
"libpq endpoint listener",
|
||||
true,
|
||||
async move {
|
||||
page_service::libpq_listener_main(conf, auth, pageserver_listener, conf.auth_type).await
|
||||
},
|
||||
);
|
||||
{
|
||||
let libpq_ctx = RequestContext::todo_child(
|
||||
TaskKind::LibpqEndpointListener,
|
||||
// listener task shouldn't need to download anything. (We will
|
||||
// create a separate sub-contexts for each connection, with their
|
||||
// own download behavior. This context is used only to listen and
|
||||
// accept connections.)
|
||||
DownloadBehavior::Error,
|
||||
);
|
||||
task_mgr::spawn(
|
||||
COMPUTE_REQUEST_RUNTIME.handle(),
|
||||
TaskKind::LibpqEndpointListener,
|
||||
None,
|
||||
None,
|
||||
"libpq endpoint listener",
|
||||
true,
|
||||
async move {
|
||||
page_service::libpq_listener_main(
|
||||
conf,
|
||||
auth,
|
||||
pageserver_listener,
|
||||
conf.auth_type,
|
||||
libpq_ctx,
|
||||
)
|
||||
.await
|
||||
},
|
||||
);
|
||||
}
|
||||
|
||||
// All started up! Now just sit and wait for shutdown signal.
|
||||
signals.handle(|signal| match signal {
|
||||
|
||||
@@ -158,6 +158,8 @@ pub struct PageServerConf {
|
||||
pub synthetic_size_calculation_interval: Duration,
|
||||
|
||||
pub test_remote_failures: u64,
|
||||
|
||||
pub ondemand_download_behavior_treat_error_as_warn: bool,
|
||||
}
|
||||
|
||||
/// We do not want to store this in a PageServerConf because the latter may be logged
|
||||
@@ -222,6 +224,8 @@ struct PageServerConfigBuilder {
|
||||
synthetic_size_calculation_interval: BuilderValue<Duration>,
|
||||
|
||||
test_remote_failures: BuilderValue<u64>,
|
||||
|
||||
ondemand_download_behavior_treat_error_as_warn: BuilderValue<bool>,
|
||||
}
|
||||
|
||||
impl Default for PageServerConfigBuilder {
|
||||
@@ -267,6 +271,8 @@ impl Default for PageServerConfigBuilder {
|
||||
metric_collection_endpoint: Set(DEFAULT_METRIC_COLLECTION_ENDPOINT),
|
||||
|
||||
test_remote_failures: Set(0),
|
||||
|
||||
ondemand_download_behavior_treat_error_as_warn: Set(false),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -363,6 +369,14 @@ impl PageServerConfigBuilder {
|
||||
self.test_remote_failures = BuilderValue::Set(fail_first);
|
||||
}
|
||||
|
||||
pub fn ondemand_download_behavior_treat_error_as_warn(
|
||||
&mut self,
|
||||
ondemand_download_behavior_treat_error_as_warn: bool,
|
||||
) {
|
||||
self.ondemand_download_behavior_treat_error_as_warn =
|
||||
BuilderValue::Set(ondemand_download_behavior_treat_error_as_warn);
|
||||
}
|
||||
|
||||
pub fn build(self) -> anyhow::Result<PageServerConf> {
|
||||
Ok(PageServerConf {
|
||||
listen_pg_addr: self
|
||||
@@ -422,6 +436,11 @@ impl PageServerConfigBuilder {
|
||||
test_remote_failures: self
|
||||
.test_remote_failures
|
||||
.ok_or(anyhow!("missing test_remote_failuers"))?,
|
||||
ondemand_download_behavior_treat_error_as_warn: self
|
||||
.ondemand_download_behavior_treat_error_as_warn
|
||||
.ok_or(anyhow!(
|
||||
"missing ondemand_download_behavior_treat_error_as_warn"
|
||||
))?,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -600,6 +619,7 @@ impl PageServerConf {
|
||||
"synthetic_size_calculation_interval" =>
|
||||
builder.synthetic_size_calculation_interval(parse_toml_duration(key, item)?),
|
||||
"test_remote_failures" => builder.test_remote_failures(parse_toml_u64(key, item)?),
|
||||
"ondemand_download_behavior_treat_error_as_warn" => builder.ondemand_download_behavior_treat_error_as_warn(parse_toml_bool(key, item)?),
|
||||
_ => bail!("unrecognized pageserver option '{key}'"),
|
||||
}
|
||||
}
|
||||
@@ -724,6 +744,7 @@ impl PageServerConf {
|
||||
metric_collection_endpoint: defaults::DEFAULT_METRIC_COLLECTION_ENDPOINT,
|
||||
synthetic_size_calculation_interval: Duration::from_secs(60),
|
||||
test_remote_failures: 0,
|
||||
ondemand_download_behavior_treat_error_as_warn: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -749,6 +770,11 @@ fn parse_toml_u64(name: &str, item: &Item) -> Result<u64> {
|
||||
Ok(i as u64)
|
||||
}
|
||||
|
||||
fn parse_toml_bool(name: &str, item: &Item) -> Result<bool> {
|
||||
item.as_bool()
|
||||
.with_context(|| format!("configure option {name} is not a bool"))
|
||||
}
|
||||
|
||||
fn parse_toml_duration(name: &str, item: &Item) -> Result<Duration> {
|
||||
let s = item
|
||||
.as_str()
|
||||
@@ -907,6 +933,7 @@ log_format = 'json'
|
||||
defaults::DEFAULT_SYNTHETIC_SIZE_CALCULATION_INTERVAL
|
||||
)?,
|
||||
test_remote_failures: 0,
|
||||
ondemand_download_behavior_treat_error_as_warn: false,
|
||||
},
|
||||
"Correct defaults should be used when no config values are provided"
|
||||
);
|
||||
@@ -954,6 +981,7 @@ log_format = 'json'
|
||||
metric_collection_endpoint: Some(Url::parse("http://localhost:80/metrics")?),
|
||||
synthetic_size_calculation_interval: Duration::from_secs(333),
|
||||
test_remote_failures: 0,
|
||||
ondemand_download_behavior_treat_error_as_warn: false,
|
||||
},
|
||||
"Should be able to parse all basic config values correctly"
|
||||
);
|
||||
|
||||
@@ -3,6 +3,7 @@
|
||||
//! and push them to a HTTP endpoint.
|
||||
//! Cache metrics to send only the updated ones.
|
||||
//!
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME};
|
||||
use crate::tenant::mgr;
|
||||
use anyhow;
|
||||
@@ -47,12 +48,15 @@ pub async fn collect_metrics(
|
||||
metric_collection_interval: Duration,
|
||||
synthetic_size_calculation_interval: Duration,
|
||||
node_id: NodeId,
|
||||
ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
let mut ticker = tokio::time::interval(metric_collection_interval);
|
||||
|
||||
info!("starting collect_metrics");
|
||||
|
||||
// spin up background worker that caclulates tenant sizes
|
||||
let worker_ctx =
|
||||
ctx.detached_child(TaskKind::CalculateSyntheticSize, DownloadBehavior::Download);
|
||||
task_mgr::spawn(
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
TaskKind::CalculateSyntheticSize,
|
||||
@@ -61,7 +65,7 @@ pub async fn collect_metrics(
|
||||
"synthetic size calculation",
|
||||
false,
|
||||
async move {
|
||||
calculate_synthetic_size_worker(synthetic_size_calculation_interval)
|
||||
calculate_synthetic_size_worker(synthetic_size_calculation_interval, &worker_ctx)
|
||||
.instrument(info_span!("synthetic_size_worker"))
|
||||
.await?;
|
||||
Ok(())
|
||||
@@ -79,7 +83,7 @@ pub async fn collect_metrics(
|
||||
return Ok(());
|
||||
},
|
||||
_ = ticker.tick() => {
|
||||
if let Err(err) = collect_metrics_iteration(&client, &mut cached_metrics, metric_collection_endpoint, node_id).await
|
||||
if let Err(err) = collect_metrics_iteration(&client, &mut cached_metrics, metric_collection_endpoint, node_id, &ctx).await
|
||||
{
|
||||
error!("metrics collection failed: {err:?}");
|
||||
}
|
||||
@@ -102,6 +106,7 @@ pub async fn collect_metrics_iteration(
|
||||
cached_metrics: &mut HashMap<PageserverConsumptionMetricsKey, u64>,
|
||||
metric_collection_endpoint: &reqwest::Url,
|
||||
node_id: NodeId,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
let mut current_metrics: Vec<(PageserverConsumptionMetricsKey, u64)> = Vec::new();
|
||||
trace!(
|
||||
@@ -137,7 +142,7 @@ pub async fn collect_metrics_iteration(
|
||||
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(ctx)?;
|
||||
// Only send timeline logical size when it is fully calculated.
|
||||
if is_exact {
|
||||
current_metrics.push((
|
||||
@@ -258,6 +263,7 @@ pub async fn collect_metrics_iteration(
|
||||
/// Caclculate synthetic size for each active tenant
|
||||
pub async fn calculate_synthetic_size_worker(
|
||||
synthetic_size_calculation_interval: Duration,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
info!("starting calculate_synthetic_size_worker");
|
||||
|
||||
@@ -280,7 +286,7 @@ pub async fn calculate_synthetic_size_worker(
|
||||
|
||||
if let Ok(tenant) = mgr::get_tenant(tenant_id, true).await
|
||||
{
|
||||
if let Err(e) = tenant.calculate_synthetic_size().await {
|
||||
if let Err(e) = tenant.calculate_synthetic_size(ctx).await {
|
||||
error!("failed to calculate synthetic size for tenant {}: {}", tenant_id, e);
|
||||
}
|
||||
}
|
||||
|
||||
199
pageserver/src/context.rs
Normal file
199
pageserver/src/context.rs
Normal file
@@ -0,0 +1,199 @@
|
||||
//! This module defines `RequestContext`, a structure that we use throughout
|
||||
//! the pageserver to propagate high-level context from places
|
||||
//! that _originate_ activity down to the shared code paths at the
|
||||
//! heart of the pageserver. It's inspired by Golang's `context.Context`.
|
||||
//!
|
||||
//! For example, in `Timeline::get(page_nr, lsn)` we need to answer the following questions:
|
||||
//! 1. What high-level activity ([`TaskKind`]) needs this page?
|
||||
//! We need that information as a categorical dimension for page access
|
||||
//! statistics, which we, in turn, need to guide layer eviction policy design.
|
||||
//! 2. How should we behave if, to produce the page image, we need to
|
||||
//! on-demand download a layer file ([`DownloadBehavior`]).
|
||||
//!
|
||||
//! [`RequestContext`] satisfies those needs.
|
||||
//! The current implementation is a small `struct` that is passed through
|
||||
//! the call chain by reference.
|
||||
//!
|
||||
//! ### Future Work
|
||||
//!
|
||||
//! However, we do not intend to stop here, since there are other needs that
|
||||
//! require carrying information from high to low levels of the app.
|
||||
//!
|
||||
//! Most importantly, **cancellation signaling** in response to
|
||||
//! 1. timeouts (page_service max response time) and
|
||||
//! 2. lifecycle requests (detach tenant, delete timeline).
|
||||
//!
|
||||
//! Related to that, there is sometimes a need to ensure that all tokio tasks spawned
|
||||
//! by the transitive callees of a request have finished. The keyword here
|
||||
//! is **Structured Concurrency**, and right now, we use `task_mgr` in most places,
|
||||
//! `TaskHandle` in some places, and careful code review around `FuturesUnordered`
|
||||
//! or `JoinSet` in other places.
|
||||
//!
|
||||
//! We do not yet have a systematic cancellation story in pageserver, and it is
|
||||
//! pretty clear that [`RequestContext`] will be responsible for that.
|
||||
//! So, the API already prepares for this role through the
|
||||
//! [`RequestContext::detached_child`] and [`RequestContext::attached_child`] methods.
|
||||
//! See their doc comments for details on how we will use them in the future.
|
||||
//!
|
||||
//! It is not clear whether or how we will enforce Structured Concurrency, and
|
||||
//! what role [`RequestContext`] will play there.
|
||||
//! So, the API doesn't prepare us for this topic.
|
||||
//!
|
||||
//! Other future uses of `RequestContext`:
|
||||
//! - Communicate compute & IO priorities (user-initiated request vs. background-loop)
|
||||
//! - Request IDs for distributed tracing
|
||||
//! - Request/Timeline/Tenant-scoped log levels
|
||||
//!
|
||||
//! RequestContext might look quite different once it supports those features.
|
||||
//! Likely, it will have a shape similar to Golang's `context.Context`.
|
||||
//!
|
||||
//! ### Why A Struct Instead Of Method Parameters
|
||||
//!
|
||||
//! What's typical about such information is that it needs to be passed down
|
||||
//! along the call chain from high level to low level, but few of the functions
|
||||
//! in the middle need to understand it.
|
||||
//! Further, it is to be expected that we will need to propagate more data
|
||||
//! in the future (see the earlier section on future work).
|
||||
//! Hence, for functions in the middle of the call chain, we have the following
|
||||
//! requirements:
|
||||
//! 1. It should be easy to forward the context to callees.
|
||||
//! 2. To propagate more data from high-level to low-level code, the functions in
|
||||
//! the middle should not need to be modified.
|
||||
//! The solution is to have a container structure ([`RequestContext`]) that
|
||||
//! carries the information. Functions that don't care about what's in it
|
||||
//! pass it along to callees.
|
||||
//!
|
||||
//! ### Why Not Task-Local Variables
|
||||
//!
|
||||
//! One could use task-local variables (the equivalent of thread-local variables)
|
||||
//! to address the immediate needs outlined above.
|
||||
//! However, we reject task-local variables because:
|
||||
//! 1. they are implicit, thereby making it harder to trace the data flow in code
|
||||
//! reviews and during debugging,
|
||||
//! 2. they can be mutable, which enables implicit return data flow,
|
||||
//! 3. they are restrictive in that code which fans out into multiple tasks,
|
||||
//! or even threads, needs to carefully propagate the state.
|
||||
//!
|
||||
//! In contrast, information flow with [`RequestContext`] is
|
||||
//! 1. always explicit,
|
||||
//! 2. strictly uni-directional because RequestContext is immutable,
|
||||
//! 3. tangible because a [`RequestContext`] is just a value.
|
||||
//! When creating child activities, regardless of whether it's a task,
|
||||
//! thread, or even an RPC to another service, the value can
|
||||
//! be used like any other argument.
|
||||
//!
|
||||
//! The solution is that all code paths are infected with precisely one
|
||||
//! [`RequestContext`] argument. Functions in the middle of the call chain
|
||||
//! only need to pass it on.
|
||||
use crate::task_mgr::TaskKind;
|
||||
|
||||
// The main structure of this module, see module-level comment.
|
||||
pub struct RequestContext {
|
||||
task_kind: TaskKind,
|
||||
download_behavior: DownloadBehavior,
|
||||
}
|
||||
|
||||
/// Desired behavior if the operation requires an on-demand download
|
||||
/// to proceed.
|
||||
#[derive(Clone, Copy, PartialEq, Eq)]
|
||||
pub enum DownloadBehavior {
|
||||
/// Download the layer file. It can take a while.
|
||||
Download,
|
||||
|
||||
/// Download the layer file, but print a warning to the log. This should be used
|
||||
/// in code where the layer file is expected to already exist locally.
|
||||
Warn,
|
||||
|
||||
/// Return a PageReconstructError::NeedsDownload error
|
||||
Error,
|
||||
}
|
||||
|
||||
impl RequestContext {
|
||||
/// Create a new RequestContext that has no parent.
|
||||
///
|
||||
/// The function is called `new` because, once we add children
|
||||
/// to it using `detached_child` or `attached_child`, the context
|
||||
/// form a tree (not implemented yet since cancellation will be
|
||||
/// the first feature that requires a tree).
|
||||
///
|
||||
/// # Future: Cancellation
|
||||
///
|
||||
/// The only reason why a context like this one can be canceled is
|
||||
/// because someone explicitly canceled it.
|
||||
/// It has no parent, so it cannot inherit cancellation from there.
|
||||
pub fn new(task_kind: TaskKind, download_behavior: DownloadBehavior) -> Self {
|
||||
RequestContext {
|
||||
task_kind,
|
||||
download_behavior,
|
||||
}
|
||||
}
|
||||
|
||||
/// Create a detached child context for a task that may outlive `self`.
|
||||
///
|
||||
/// Use this when spawning new background activity that should complete
|
||||
/// even if the current request is canceled.
|
||||
///
|
||||
/// # Future: Cancellation
|
||||
///
|
||||
/// Cancellation of `self` will not propagate to the child context returned
|
||||
/// by this method.
|
||||
///
|
||||
/// # Future: Structured Concurrency
|
||||
///
|
||||
/// We could add the Future as a parameter to this function, spawn it as a task,
|
||||
/// and pass to the new task the child context as an argument.
|
||||
/// That would be an ergonomic improvement.
|
||||
///
|
||||
/// We could make new calls to this function fail if `self` is already canceled.
|
||||
pub fn detached_child(&self, task_kind: TaskKind, download_behavior: DownloadBehavior) -> Self {
|
||||
self.child_impl(task_kind, download_behavior)
|
||||
}
|
||||
|
||||
/// Create a child of context `self` for a task that shall not outlive `self`.
|
||||
///
|
||||
/// Use this when fanning-out work to other async tasks.
|
||||
///
|
||||
/// # Future: Cancellation
|
||||
///
|
||||
/// Cancelling a context will propagate to its attached children.
|
||||
///
|
||||
/// # Future: Structured Concurrency
|
||||
///
|
||||
/// We could add the Future as a parameter to this function, spawn it as a task,
|
||||
/// and track its `JoinHandle` inside the `RequestContext`.
|
||||
///
|
||||
/// We could then provide another method to allow waiting for all child tasks
|
||||
/// to finish.
|
||||
///
|
||||
/// We could make new calls to this function fail if `self` is already canceled.
|
||||
/// Alternatively, we could allow the creation but not spawn the task.
|
||||
/// The method to wait for child tasks would return an error, indicating
|
||||
/// that the child task was not started because the context was canceled.
|
||||
pub fn attached_child(&self) -> Self {
|
||||
self.child_impl(self.task_kind(), self.download_behavior())
|
||||
}
|
||||
|
||||
/// Use this function when you should be creating a child context using
|
||||
/// [`attached_child`] or [`detached_child`], but your caller doesn't provide
|
||||
/// a context and you are unwilling to change all callers to provide one.
|
||||
///
|
||||
/// Before we add cancellation, we should get rid of this method.
|
||||
pub fn todo_child(task_kind: TaskKind, download_behavior: DownloadBehavior) -> Self {
|
||||
Self::new(task_kind, download_behavior)
|
||||
}
|
||||
|
||||
fn child_impl(&self, task_kind: TaskKind, download_behavior: DownloadBehavior) -> Self {
|
||||
RequestContext {
|
||||
task_kind,
|
||||
download_behavior,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn task_kind(&self) -> TaskKind {
|
||||
self.task_kind
|
||||
}
|
||||
|
||||
pub fn download_behavior(&self) -> DownloadBehavior {
|
||||
self.download_behavior
|
||||
}
|
||||
}
|
||||
@@ -12,7 +12,9 @@ use super::models::{
|
||||
StatusResponse, TenantConfigRequest, TenantCreateRequest, TenantCreateResponse, TenantInfo,
|
||||
TimelineCreateRequest, TimelineInfo,
|
||||
};
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::pgdatadir_mapping::LsnForTimestamp;
|
||||
use crate::task_mgr::TaskKind;
|
||||
use crate::tenant::config::TenantConfOpt;
|
||||
use crate::tenant::{PageReconstructError, Timeline};
|
||||
use crate::{config::PageServerConf, tenant::mgr};
|
||||
@@ -81,6 +83,16 @@ fn check_permission(request: &Request<Body>, tenant_id: Option<TenantId>) -> Res
|
||||
fn apierror_from_prerror(err: PageReconstructError) -> ApiError {
|
||||
match err {
|
||||
PageReconstructError::Other(err) => ApiError::InternalServerError(err),
|
||||
PageReconstructError::NeedsDownload(_, _) => {
|
||||
// This shouldn't happen, because we use a RequestContext that requests to
|
||||
// download any missing layer files on-demand.
|
||||
ApiError::InternalServerError(anyhow::anyhow!(
|
||||
"would need to download remote layer file"
|
||||
))
|
||||
}
|
||||
PageReconstructError::Cancelled => {
|
||||
ApiError::InternalServerError(anyhow::anyhow!("request was cancelled"))
|
||||
}
|
||||
PageReconstructError::WalRedo(err) => {
|
||||
ApiError::InternalServerError(anyhow::Error::new(err))
|
||||
}
|
||||
@@ -91,8 +103,9 @@ fn apierror_from_prerror(err: PageReconstructError) -> ApiError {
|
||||
async fn build_timeline_info(
|
||||
timeline: &Arc<Timeline>,
|
||||
include_non_incremental_logical_size: bool,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<TimelineInfo> {
|
||||
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
|
||||
@@ -102,6 +115,7 @@ async fn build_timeline_info(
|
||||
.get_current_logical_size_non_incremental(
|
||||
info.last_record_lsn,
|
||||
CancellationToken::new(),
|
||||
ctx,
|
||||
)
|
||||
.await?,
|
||||
);
|
||||
@@ -109,7 +123,10 @@ async fn build_timeline_info(
|
||||
Ok(info)
|
||||
}
|
||||
|
||||
fn build_timeline_info_common(timeline: &Arc<Timeline>) -> anyhow::Result<TimelineInfo> {
|
||||
fn build_timeline_info_common(
|
||||
timeline: &Arc<Timeline>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<TimelineInfo> {
|
||||
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();
|
||||
@@ -129,7 +146,7 @@ fn build_timeline_info_common(timeline: &Arc<Timeline>) -> anyhow::Result<Timeli
|
||||
Lsn(0) => 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:?}");
|
||||
@@ -180,6 +197,8 @@ async fn timeline_create_handler(mut request: Request<Body>) -> Result<Response<
|
||||
.new_timeline_id
|
||||
.unwrap_or_else(TimelineId::generate);
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Error);
|
||||
|
||||
let tenant = mgr::get_tenant(tenant_id, true)
|
||||
.await
|
||||
.map_err(ApiError::NotFound)?;
|
||||
@@ -187,13 +206,14 @@ async fn timeline_create_handler(mut request: Request<Body>) -> Result<Response<
|
||||
new_timeline_id,
|
||||
request_data.ancestor_timeline_id.map(TimelineId::from),
|
||||
request_data.ancestor_start_lsn,
|
||||
request_data.pg_version.unwrap_or(crate::DEFAULT_PG_VERSION)
|
||||
request_data.pg_version.unwrap_or(crate::DEFAULT_PG_VERSION),
|
||||
&ctx,
|
||||
)
|
||||
.instrument(info_span!("timeline_create", tenant = %tenant_id, new_timeline = ?request_data.new_timeline_id, timeline_id = %new_timeline_id, lsn=?request_data.ancestor_start_lsn, pg_version=?request_data.pg_version))
|
||||
.await {
|
||||
Ok(Some(new_timeline)) => {
|
||||
// Created. Construct a TimelineInfo for it.
|
||||
let timeline_info = build_timeline_info_common(&new_timeline)
|
||||
let timeline_info = build_timeline_info_common(&new_timeline, &ctx)
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
json_response(StatusCode::CREATED, timeline_info)
|
||||
}
|
||||
@@ -208,6 +228,8 @@ async fn timeline_list_handler(request: Request<Body>) -> Result<Response<Body>,
|
||||
query_param_present(&request, "include-non-incremental-logical-size");
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
|
||||
let response_data = async {
|
||||
let tenant = mgr::get_tenant(tenant_id, true)
|
||||
.await
|
||||
@@ -217,7 +239,7 @@ async fn timeline_list_handler(request: Request<Body>) -> Result<Response<Body>,
|
||||
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)
|
||||
build_timeline_info(&timeline, include_non_incremental_logical_size, &ctx)
|
||||
.await
|
||||
.context(
|
||||
"Failed to convert tenant timeline {timeline_id} into the local one: {e:?}",
|
||||
@@ -266,6 +288,9 @@ async fn timeline_detail_handler(request: Request<Body>) -> Result<Response<Body
|
||||
query_param_present(&request, "include-non-incremental-logical-size");
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
// Logical size calculation needs downloading.
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
|
||||
let timeline_info = async {
|
||||
let tenant = mgr::get_tenant(tenant_id, true)
|
||||
.await
|
||||
@@ -275,10 +300,11 @@ async fn timeline_detail_handler(request: Request<Body>) -> Result<Response<Body
|
||||
.get_timeline(timeline_id, false)
|
||||
.map_err(ApiError::NotFound)?;
|
||||
|
||||
let timeline_info = build_timeline_info(&timeline, include_non_incremental_logical_size)
|
||||
.await
|
||||
.context("get local timeline info")
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
let timeline_info =
|
||||
build_timeline_info(&timeline, include_non_incremental_logical_size, &ctx)
|
||||
.await
|
||||
.context("get local timeline info")
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
Ok::<_, ApiError>(timeline_info)
|
||||
}
|
||||
@@ -299,12 +325,13 @@ async fn get_lsn_by_timestamp_handler(request: Request<Body>) -> Result<Response
|
||||
.map_err(ApiError::BadRequest)?;
|
||||
let timestamp_pg = postgres_ffi::to_pg_timestamp(timestamp);
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
let timeline = mgr::get_tenant(tenant_id, true)
|
||||
.await
|
||||
.and_then(|tenant| tenant.get_timeline(timeline_id, true))
|
||||
.map_err(ApiError::NotFound)?;
|
||||
let result = timeline
|
||||
.find_lsn_for_timestamp(timestamp_pg)
|
||||
.find_lsn_for_timestamp(timestamp_pg, &ctx)
|
||||
.await
|
||||
.map_err(apierror_from_prerror)?;
|
||||
|
||||
@@ -322,13 +349,15 @@ async fn tenant_attach_handler(request: Request<Body>) -> Result<Response<Body>,
|
||||
let tenant_id: TenantId = parse_request_param(&request, "tenant_id")?;
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn);
|
||||
|
||||
info!("Handling tenant attach {tenant_id}");
|
||||
|
||||
let state = get_state(&request);
|
||||
|
||||
if let Some(remote_storage) = &state.remote_storage {
|
||||
// FIXME: distinguish between "Tenant already exists" and other errors
|
||||
mgr::attach_tenant(state.conf, tenant_id, remote_storage.clone())
|
||||
mgr::attach_tenant(state.conf, tenant_id, remote_storage.clone(), &ctx)
|
||||
.instrument(info_span!("tenant_attach", tenant = %tenant_id))
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
@@ -346,7 +375,9 @@ async fn timeline_delete_handler(request: Request<Body>) -> Result<Response<Body
|
||||
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
mgr::delete_timeline(tenant_id, timeline_id)
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn);
|
||||
|
||||
mgr::delete_timeline(tenant_id, timeline_id, &ctx)
|
||||
.instrument(info_span!("timeline_delete", tenant = %tenant_id, timeline = %timeline_id))
|
||||
.await
|
||||
// FIXME: Errors from `delete_timeline` can occur for a number of reasons, incuding both
|
||||
@@ -377,8 +408,10 @@ async fn tenant_load_handler(request: Request<Body>) -> Result<Response<Body>, A
|
||||
let tenant_id: TenantId = parse_request_param(&request, "tenant_id")?;
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn);
|
||||
|
||||
let state = get_state(&request);
|
||||
mgr::load_tenant(state.conf, tenant_id, state.remote_storage.clone())
|
||||
mgr::load_tenant(state.conf, tenant_id, state.remote_storage.clone(), &ctx)
|
||||
.instrument(info_span!("load", tenant = %tenant_id))
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
@@ -466,13 +499,14 @@ async fn tenant_size_handler(request: Request<Body>) -> Result<Response<Body>, A
|
||||
false
|
||||
};
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
let tenant = mgr::get_tenant(tenant_id, true)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
// this can be long operation
|
||||
let inputs = tenant
|
||||
.gather_size_inputs()
|
||||
.gather_size_inputs(&ctx)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -521,6 +555,8 @@ fn bad_duration<'a>(field_name: &'static str, value: &'a str) -> impl 'a + Fn()
|
||||
async fn tenant_create_handler(mut request: Request<Body>) -> Result<Response<Body>, ApiError> {
|
||||
check_permission(&request, None)?;
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn);
|
||||
|
||||
let request_data: TenantCreateRequest = json_request(&mut request).await?;
|
||||
|
||||
let mut tenant_conf = TenantConfOpt::default();
|
||||
@@ -598,6 +634,7 @@ async fn tenant_create_handler(mut request: Request<Body>) -> Result<Response<Bo
|
||||
tenant_conf,
|
||||
target_tenant_id,
|
||||
state.remote_storage.clone(),
|
||||
&ctx,
|
||||
)
|
||||
.instrument(info_span!("tenant_create", tenant = ?target_tenant_id))
|
||||
.await
|
||||
@@ -747,7 +784,8 @@ async fn timeline_gc_handler(mut request: Request<Body>) -> Result<Response<Body
|
||||
|
||||
let gc_req: TimelineGcRequest = json_request(&mut request).await?;
|
||||
|
||||
let wait_task_done = mgr::immediate_gc(tenant_id, timeline_id, gc_req).await?;
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
let wait_task_done = mgr::immediate_gc(tenant_id, timeline_id, gc_req, &ctx).await?;
|
||||
let gc_result = wait_task_done
|
||||
.await
|
||||
.context("wait for gc task")
|
||||
@@ -764,7 +802,8 @@ async fn timeline_compact_handler(request: Request<Body>) -> Result<Response<Bod
|
||||
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
let result_receiver = mgr::immediate_compact(tenant_id, timeline_id)
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
let result_receiver = mgr::immediate_compact(tenant_id, timeline_id, &ctx)
|
||||
.await
|
||||
.context("spawn compaction task")
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
@@ -785,6 +824,7 @@ async fn timeline_checkpoint_handler(request: Request<Body>) -> Result<Response<
|
||||
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
|
||||
check_permission(&request, Some(tenant_id))?;
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
|
||||
let tenant = mgr::get_tenant(tenant_id, true)
|
||||
.await
|
||||
.map_err(ApiError::NotFound)?;
|
||||
@@ -796,7 +836,7 @@ async fn timeline_checkpoint_handler(request: Request<Body>) -> Result<Response<
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
timeline
|
||||
.compact()
|
||||
.compact(&ctx)
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
|
||||
@@ -12,6 +12,7 @@ use tokio_tar::Archive;
|
||||
use tracing::*;
|
||||
use walkdir::WalkDir;
|
||||
|
||||
use crate::context::RequestContext;
|
||||
use crate::pgdatadir_mapping::*;
|
||||
use crate::tenant::Timeline;
|
||||
use crate::walingest::WalIngest;
|
||||
@@ -47,6 +48,7 @@ pub async fn import_timeline_from_postgres_datadir(
|
||||
tline: &Timeline,
|
||||
pgdata_path: &Path,
|
||||
pgdata_lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<()> {
|
||||
let mut pg_control: Option<ControlFileData> = None;
|
||||
|
||||
@@ -69,7 +71,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 +101,7 @@ pub async fn import_timeline_from_postgres_datadir(
|
||||
tline,
|
||||
Lsn(pg_control.checkPointCopy.redo),
|
||||
pgdata_lsn,
|
||||
ctx,
|
||||
)
|
||||
.await?;
|
||||
|
||||
@@ -113,6 +116,7 @@ async fn import_rel(
|
||||
dboid: Oid,
|
||||
reader: &mut (impl AsyncRead + Send + Sync + Unpin),
|
||||
len: usize,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// Does it look like a relation file?
|
||||
trace!("importing rel file {}", path.display());
|
||||
@@ -147,7 +151,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 +189,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 +202,7 @@ async fn import_slru(
|
||||
path: &Path,
|
||||
reader: &mut (impl AsyncRead + Send + Sync + Unpin),
|
||||
len: usize,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
info!("importing slru file {path:?}");
|
||||
|
||||
@@ -211,7 +219,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,15 +260,15 @@ async fn import_wal(
|
||||
tline: &Timeline,
|
||||
startpoint: Lsn,
|
||||
endpoint: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
use std::io::Read;
|
||||
let mut waldecoder = WalStreamDecoder::new(startpoint, tline.pg_version);
|
||||
|
||||
let mut segno = startpoint.segment_number(WAL_SEGMENT_SIZE);
|
||||
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
|
||||
@@ -283,6 +291,7 @@ async fn import_wal(
|
||||
file.seek(std::io::SeekFrom::Start(offset as u64))?;
|
||||
}
|
||||
|
||||
use std::io::Read;
|
||||
let nread = file.read_to_end(&mut buf)?;
|
||||
if nread != WAL_SEGMENT_SIZE - offset {
|
||||
// Maybe allow this for .partial files?
|
||||
@@ -297,7 +306,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 +335,7 @@ pub async fn import_basebackup_from_tar(
|
||||
tline: &Timeline,
|
||||
reader: &mut (impl AsyncRead + Send + Sync + Unpin),
|
||||
base_lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<()> {
|
||||
info!("importing base at {base_lsn}");
|
||||
let mut modification = tline.begin_modification(base_lsn);
|
||||
@@ -344,7 +354,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 +386,14 @@ pub async fn import_wal_from_tar(
|
||||
reader: &mut (impl AsyncRead + Send + Sync + Unpin),
|
||||
start_lsn: Lsn,
|
||||
end_lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +442,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 +477,7 @@ async fn import_file(
|
||||
file_path: &Path,
|
||||
reader: &mut (impl AsyncRead + Send + Sync + Unpin),
|
||||
len: usize,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Option<ControlFileData>> {
|
||||
let file_name = match file_path.file_name() {
|
||||
Some(name) => name.to_string_lossy(),
|
||||
@@ -498,14 +510,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 +535,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") {
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -43,6 +43,7 @@ use utils::{
|
||||
use crate::auth::check_permission;
|
||||
use crate::basebackup;
|
||||
use crate::config::PageServerConf;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::import_datadir::import_wal_from_tar;
|
||||
use crate::metrics::{LIVE_CONNECTIONS_COUNT, SMGR_QUERY_TIME};
|
||||
use crate::task_mgr;
|
||||
@@ -123,6 +124,7 @@ pub async fn libpq_listener_main(
|
||||
auth: Option<Arc<JwtAuth>>,
|
||||
listener: TcpListener,
|
||||
auth_type: AuthType,
|
||||
listener_ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
listener.set_nonblocking(true)?;
|
||||
let tokio_listener = tokio::net::TcpListener::from_std(listener)?;
|
||||
@@ -146,6 +148,9 @@ pub async fn libpq_listener_main(
|
||||
debug!("accepted connection from {}", peer_addr);
|
||||
let local_auth = auth.clone();
|
||||
|
||||
let connection_ctx = listener_ctx
|
||||
.detached_child(TaskKind::PageRequestHandler, DownloadBehavior::Download);
|
||||
|
||||
// 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 +162,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 +182,7 @@ async fn page_service_conn_main(
|
||||
auth: Option<Arc<JwtAuth>>,
|
||||
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 +197,11 @@ async fn page_service_conn_main(
|
||||
.set_nodelay(true)
|
||||
.context("could not set TCP_NODELAY")?;
|
||||
|
||||
let mut conn_handler = PageServerHandler::new(conf, auth);
|
||||
// XXX: pgbackend.run() should take the connection_ctx,
|
||||
// and create a child per-query context when it invokes process_query.
|
||||
// But it's in a shared crate, so, we store connection_ctx inside PageServerHandler
|
||||
// and create the per-query context in process_query ourselves.
|
||||
let mut conn_handler = PageServerHandler::new(conf, auth, connection_ctx);
|
||||
let pgbackend = PostgresBackend::new(socket, auth_type, None)?;
|
||||
|
||||
match pgbackend
|
||||
@@ -251,30 +261,42 @@ struct PageServerHandler {
|
||||
_conf: &'static PageServerConf,
|
||||
auth: Option<Arc<JwtAuth>>,
|
||||
claims: Option<Claims>,
|
||||
|
||||
/// The context created for the lifetime of the connection
|
||||
/// services by this PageServerHandler.
|
||||
/// For each query received over the connection,
|
||||
/// `process_query` creates a child context from this one.
|
||||
connection_ctx: RequestContext,
|
||||
}
|
||||
|
||||
impl PageServerHandler {
|
||||
pub fn new(conf: &'static PageServerConf, auth: Option<Arc<JwtAuth>>) -> Self {
|
||||
pub fn new(
|
||||
conf: &'static PageServerConf,
|
||||
auth: Option<Arc<JwtAuth>>,
|
||||
connection_ctx: RequestContext,
|
||||
) -> Self {
|
||||
PageServerHandler {
|
||||
_conf: conf,
|
||||
auth,
|
||||
claims: None,
|
||||
connection_ctx,
|
||||
}
|
||||
}
|
||||
|
||||
#[instrument(skip(self, pgb))]
|
||||
#[instrument(skip(self, pgb, ctx))]
|
||||
async fn handle_pagerequests(
|
||||
&self,
|
||||
pgb: &mut PostgresBackend,
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// NOTE: pagerequests handler exits when connection is closed,
|
||||
// so there is no need to reset the association
|
||||
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, &ctx).await?;
|
||||
let mut tracer = if tenant.get_trace_read_requests() {
|
||||
let connection_id = ConnectionId::generate();
|
||||
let path = tenant
|
||||
@@ -325,22 +347,27 @@ impl PageServerHandler {
|
||||
|
||||
let neon_fe_msg = PagestreamFeMessage::parse(&mut copy_data_bytes.reader())?;
|
||||
|
||||
// TODO: We could create a new per-request context here, with unique ID.
|
||||
// Currently we use the same per-timeline context for all requests
|
||||
|
||||
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
|
||||
}
|
||||
};
|
||||
|
||||
@@ -359,7 +386,8 @@ impl PageServerHandler {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[instrument(skip(self, pgb))]
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
#[instrument(skip(self, pgb, ctx))]
|
||||
async fn handle_import_basebackup(
|
||||
&self,
|
||||
pgb: &mut PostgresBackend,
|
||||
@@ -368,12 +396,13 @@ impl PageServerHandler {
|
||||
base_lsn: Lsn,
|
||||
_end_lsn: Lsn,
|
||||
pg_version: u32,
|
||||
ctx: RequestContext,
|
||||
) -> Result<(), QueryError> {
|
||||
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, &ctx).await?;
|
||||
let timeline = tenant.create_empty_timeline(timeline_id, base_lsn, pg_version, &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
|
||||
@@ -392,7 +421,7 @@ impl PageServerHandler {
|
||||
|
||||
let mut copyin_stream = Box::pin(copyin_stream(pgb));
|
||||
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
|
||||
@@ -414,7 +443,7 @@ impl PageServerHandler {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[instrument(skip(self, pgb))]
|
||||
#[instrument(skip(self, pgb, ctx))]
|
||||
async fn handle_import_wal(
|
||||
&self,
|
||||
pgb: &mut PostgresBackend,
|
||||
@@ -422,10 +451,11 @@ impl PageServerHandler {
|
||||
timeline_id: TimelineId,
|
||||
start_lsn: Lsn,
|
||||
end_lsn: Lsn,
|
||||
ctx: RequestContext,
|
||||
) -> Result<(), QueryError> {
|
||||
task_mgr::associate_with(Some(tenant_id), Some(timeline_id));
|
||||
|
||||
let timeline = get_active_tenant_timeline(tenant_id, timeline_id).await?;
|
||||
let timeline = get_active_tenant_timeline(tenant_id, timeline_id, &ctx).await?;
|
||||
let last_record_lsn = timeline.get_last_record_lsn();
|
||||
if last_record_lsn != start_lsn {
|
||||
return Err(QueryError::Other(
|
||||
@@ -442,7 +472,7 @@ impl PageServerHandler {
|
||||
pgb.flush().await?;
|
||||
let mut copyin_stream = Box::pin(copyin_stream(pgb));
|
||||
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
|
||||
@@ -488,6 +518,7 @@ impl PageServerHandler {
|
||||
mut lsn: Lsn,
|
||||
latest: bool,
|
||||
latest_gc_cutoff_lsn: &RcuReadGuard<Lsn>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Lsn> {
|
||||
if latest {
|
||||
// Latest page version was requested. If LSN is given, it is a hint
|
||||
@@ -511,7 +542,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
|
||||
@@ -521,7 +552,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,
|
||||
@@ -531,52 +562,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: &RequestContext,
|
||||
) -> anyhow::Result<PagestreamBeMessage> {
|
||||
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: &RequestContext,
|
||||
) -> anyhow::Result<PagestreamBeMessage> {
|
||||
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: &RequestContext,
|
||||
) -> anyhow::Result<PagestreamBeMessage> {
|
||||
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;
|
||||
|
||||
@@ -585,15 +624,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: &RequestContext,
|
||||
) -> anyhow::Result<PagestreamBeMessage> {
|
||||
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.
|
||||
@@ -604,7 +645,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 {
|
||||
@@ -612,23 +653,25 @@ impl PageServerHandler {
|
||||
}))
|
||||
}
|
||||
|
||||
#[instrument(skip(self, pgb))]
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
#[instrument(skip(self, pgb, ctx))]
|
||||
async fn handle_basebackup_request(
|
||||
&self,
|
||||
&mut self,
|
||||
pgb: &mut PostgresBackend,
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
lsn: Option<Lsn>,
|
||||
prev_lsn: Option<Lsn>,
|
||||
full_backup: bool,
|
||||
ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// check that the timeline exists
|
||||
let timeline = get_active_tenant_timeline(tenant_id, timeline_id).await?;
|
||||
let timeline = get_active_tenant_timeline(tenant_id, timeline_id, &ctx).await?;
|
||||
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")?;
|
||||
@@ -641,8 +684,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)?;
|
||||
@@ -713,6 +763,7 @@ impl postgres_backend_async::Handler for PageServerHandler {
|
||||
pgb: &mut PostgresBackend,
|
||||
query_string: &str,
|
||||
) -> Result<(), QueryError> {
|
||||
let ctx = self.connection_ctx.attached_child();
|
||||
debug!("process query {query_string:?}");
|
||||
|
||||
if query_string.starts_with("pagestream ") {
|
||||
@@ -730,7 +781,7 @@ impl postgres_backend_async::Handler for PageServerHandler {
|
||||
|
||||
self.check_permission(Some(tenant_id))?;
|
||||
|
||||
self.handle_pagerequests(pgb, tenant_id, timeline_id)
|
||||
self.handle_pagerequests(pgb, tenant_id, timeline_id, ctx)
|
||||
.await?;
|
||||
} else if query_string.starts_with("basebackup ") {
|
||||
let (_, params_raw) = query_string.split_at("basebackup ".len());
|
||||
@@ -759,7 +810,7 @@ impl postgres_backend_async::Handler for PageServerHandler {
|
||||
};
|
||||
|
||||
// Check that the timeline exists
|
||||
self.handle_basebackup_request(pgb, tenant_id, timeline_id, lsn, None, false)
|
||||
self.handle_basebackup_request(pgb, tenant_id, timeline_id, lsn, None, false, ctx)
|
||||
.await?;
|
||||
pgb.write_message(&BeMessage::CommandComplete(b"SELECT 1"))?;
|
||||
}
|
||||
@@ -780,7 +831,7 @@ 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_tenant_timeline(tenant_id, timeline_id).await?;
|
||||
let timeline = get_active_tenant_timeline(tenant_id, timeline_id, &ctx).await?;
|
||||
|
||||
let end_of_timeline = timeline.get_last_record_rlsn();
|
||||
|
||||
@@ -831,7 +882,7 @@ impl postgres_backend_async::Handler for PageServerHandler {
|
||||
self.check_permission(Some(tenant_id))?;
|
||||
|
||||
// Check that the timeline exists
|
||||
self.handle_basebackup_request(pgb, tenant_id, timeline_id, lsn, prev_lsn, true)
|
||||
self.handle_basebackup_request(pgb, tenant_id, timeline_id, lsn, prev_lsn, true, ctx)
|
||||
.await?;
|
||||
pgb.write_message(&BeMessage::CommandComplete(b"SELECT 1"))?;
|
||||
} else if query_string.starts_with("import basebackup ") {
|
||||
@@ -874,6 +925,7 @@ impl postgres_backend_async::Handler for PageServerHandler {
|
||||
base_lsn,
|
||||
end_lsn,
|
||||
pg_version,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
{
|
||||
@@ -910,7 +962,7 @@ impl postgres_backend_async::Handler for PageServerHandler {
|
||||
self.check_permission(Some(tenant_id))?;
|
||||
|
||||
match self
|
||||
.handle_import_wal(pgb, tenant_id, timeline_id, start_lsn, end_lsn)
|
||||
.handle_import_wal(pgb, tenant_id, timeline_id, start_lsn, end_lsn, ctx)
|
||||
.await
|
||||
{
|
||||
Ok(()) => pgb.write_message(&BeMessage::CommandComplete(b"SELECT 1"))?,
|
||||
@@ -940,7 +992,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, &ctx).await?;
|
||||
pgb.write_message(&BeMessage::RowDescription(&[
|
||||
RowDescriptor::int8_col(b"checkpoint_distance"),
|
||||
RowDescriptor::int8_col(b"checkpoint_timeout"),
|
||||
@@ -1017,6 +1069,7 @@ impl From<GetActiveTenantError> for QueryError {
|
||||
/// all tenants are still loading.
|
||||
async fn get_active_tenant_with_timeout(
|
||||
tenant_id: TenantId,
|
||||
_ctx: &RequestContext, /* require get a context to support cancellation in the future */
|
||||
) -> Result<Arc<Tenant>, GetActiveTenantError> {
|
||||
let tenant = mgr::get_tenant(tenant_id, false).await?;
|
||||
let wait_time = Duration::from_secs(30);
|
||||
@@ -1042,8 +1095,9 @@ async fn get_active_tenant_with_timeout(
|
||||
async fn get_active_tenant_timeline(
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Arc<Timeline>, GetActiveTenantError> {
|
||||
let tenant = get_active_tenant_with_timeout(tenant_id).await?;
|
||||
let tenant = get_active_tenant_with_timeout(tenant_id, ctx).await?;
|
||||
let timeline = tenant.get_timeline(timeline_id, true)?;
|
||||
Ok(timeline)
|
||||
}
|
||||
|
||||
@@ -7,6 +7,7 @@
|
||||
//! Clarify that)
|
||||
//!
|
||||
use super::tenant::{PageReconstructError, Timeline};
|
||||
use crate::context::RequestContext;
|
||||
use crate::keyspace::{KeySpace, KeySpaceAccum};
|
||||
use crate::repository::*;
|
||||
use crate::walrecord::NeonWalRecord;
|
||||
@@ -97,6 +98,7 @@ impl Timeline {
|
||||
blknum: BlockNumber,
|
||||
lsn: Lsn,
|
||||
latest: bool,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
if tag.relnode == 0 {
|
||||
return Err(PageReconstructError::Other(anyhow::anyhow!(
|
||||
@@ -104,7 +106,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 +116,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 +126,14 @@ impl Timeline {
|
||||
dbnode: Oid,
|
||||
lsn: Lsn,
|
||||
latest: bool,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<usize, PageReconstructError> {
|
||||
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 +145,7 @@ impl Timeline {
|
||||
tag: RelTag,
|
||||
lsn: Lsn,
|
||||
latest: bool,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<BlockNumber, PageReconstructError> {
|
||||
if tag.relnode == 0 {
|
||||
return Err(PageReconstructError::Other(anyhow::anyhow!(
|
||||
@@ -154,7 +158,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 +168,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 +190,7 @@ impl Timeline {
|
||||
tag: RelTag,
|
||||
lsn: Lsn,
|
||||
_latest: bool,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<bool, PageReconstructError> {
|
||||
if tag.relnode == 0 {
|
||||
return Err(PageReconstructError::Other(anyhow::anyhow!(
|
||||
@@ -199,7 +204,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 +221,11 @@ impl Timeline {
|
||||
spcnode: Oid,
|
||||
dbnode: Oid,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<HashSet<RelTag>, 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 +250,10 @@ impl Timeline {
|
||||
segno: u32,
|
||||
blknum: BlockNumber,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
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 +262,10 @@ impl Timeline {
|
||||
kind: SlruKind,
|
||||
segno: u32,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<BlockNumber, PageReconstructError> {
|
||||
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 +275,11 @@ impl Timeline {
|
||||
kind: SlruKind,
|
||||
segno: u32,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<bool, PageReconstructError> {
|
||||
// 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 +300,7 @@ impl Timeline {
|
||||
pub async fn find_lsn_for_timestamp(
|
||||
&self,
|
||||
search_timestamp: TimestampTz,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<LsnForTimestamp, PageReconstructError> {
|
||||
let gc_cutoff_lsn_guard = self.get_latest_gc_cutoff_lsn();
|
||||
let min_lsn = *gc_cutoff_lsn_guard;
|
||||
@@ -313,6 +323,7 @@ impl Timeline {
|
||||
Lsn(mid * 8),
|
||||
&mut found_smaller,
|
||||
&mut found_larger,
|
||||
ctx,
|
||||
)
|
||||
.await?;
|
||||
|
||||
@@ -362,14 +373,18 @@ impl Timeline {
|
||||
probe_lsn: Lsn,
|
||||
found_smaller: &mut bool,
|
||||
found_larger: &mut bool,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<bool, PageReconstructError> {
|
||||
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 +409,12 @@ impl Timeline {
|
||||
&self,
|
||||
kind: SlruKind,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<HashSet<u32>, 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 +426,21 @@ impl Timeline {
|
||||
spcnode: Oid,
|
||||
dbnode: Oid,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
let key = relmap_file_key(spcnode, dbnode);
|
||||
|
||||
self.get(key, lsn).await
|
||||
let buf = self.get(key, lsn, ctx).await?;
|
||||
Ok(buf)
|
||||
}
|
||||
|
||||
pub async fn list_dbdirs(
|
||||
&self,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<HashMap<(Oid, Oid), bool>, 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 +452,20 @@ impl Timeline {
|
||||
&self,
|
||||
xid: TransactionId,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
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: &RequestContext,
|
||||
) -> Result<HashSet<TransactionId>, 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 +473,20 @@ impl Timeline {
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn get_control_file(&self, lsn: Lsn) -> Result<Bytes, PageReconstructError> {
|
||||
self.get(CONTROLFILE_KEY, lsn).await
|
||||
pub async fn get_control_file(
|
||||
&self,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
self.get(CONTROLFILE_KEY, lsn, ctx).await
|
||||
}
|
||||
|
||||
pub async fn get_checkpoint(&self, lsn: Lsn) -> Result<Bytes, PageReconstructError> {
|
||||
self.get(CHECKPOINT_KEY, lsn).await
|
||||
pub async fn get_checkpoint(
|
||||
&self,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
self.get(CHECKPOINT_KEY, lsn, ctx).await
|
||||
}
|
||||
|
||||
/// Does the same as get_current_logical_size but counted on demand.
|
||||
@@ -469,15 +498,16 @@ impl Timeline {
|
||||
&self,
|
||||
lsn: Lsn,
|
||||
cancel: CancellationToken,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<u64, CalculateLogicalSizeError> {
|
||||
// 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 +516,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
|
||||
.with_context(|| format!("read relation size of {rel:?}"))?;
|
||||
let relsize = buf.get_u32_le();
|
||||
@@ -501,7 +531,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<KeySpace> {
|
||||
pub async fn collect_keyspace(
|
||||
&self,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<KeySpace> {
|
||||
// Iterate through key ranges, greedily packing them into partitions
|
||||
let mut result = KeySpaceAccum::new();
|
||||
|
||||
@@ -509,7 +543,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 +553,14 @@ impl Timeline {
|
||||
result.add_key(rel_dir_to_key(spcnode, dbnode));
|
||||
|
||||
let mut rels: Vec<RelTag> = 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 +576,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<u32> = 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 +594,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<TransactionId> = twophase_dir.xids.iter().cloned().collect();
|
||||
xids.sort_unstable();
|
||||
@@ -723,9 +757,10 @@ impl<'a> DatadirModification<'a> {
|
||||
spcnode: Oid,
|
||||
dbnode: Oid,
|
||||
img: Bytes,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +790,10 @@ impl<'a> DatadirModification<'a> {
|
||||
&mut self,
|
||||
xid: TransactionId,
|
||||
img: Bytes,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +817,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: &RequestContext,
|
||||
) -> 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 +858,12 @@ impl<'a> DatadirModification<'a> {
|
||||
&mut self,
|
||||
rel: RelTag,
|
||||
nblocks: BlockNumber,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +875,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 +907,14 @@ impl<'a> DatadirModification<'a> {
|
||||
&mut self,
|
||||
rel: RelTag,
|
||||
nblocks: BlockNumber,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +938,13 @@ impl<'a> DatadirModification<'a> {
|
||||
&mut self,
|
||||
rel: RelTag,
|
||||
nblocks: BlockNumber,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +960,12 @@ 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: &RequestContext) -> 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 +976,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 +993,11 @@ impl<'a> DatadirModification<'a> {
|
||||
kind: SlruKind,
|
||||
segno: u32,
|
||||
nblocks: BlockNumber,
|
||||
ctx: &RequestContext,
|
||||
) -> 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 +1033,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: &RequestContext,
|
||||
) -> 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 +1065,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: &RequestContext,
|
||||
) -> 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 +1165,7 @@ impl<'a> DatadirModification<'a> {
|
||||
|
||||
// Internal helper functions to batch the modifications
|
||||
|
||||
async fn get(&self, key: Key) -> Result<Bytes, PageReconstructError> {
|
||||
async fn get(&self, key: Key, ctx: &RequestContext) -> Result<Bytes, PageReconstructError> {
|
||||
// Have we already updated the same key? Read the pending updated
|
||||
// version in that case.
|
||||
//
|
||||
@@ -1132,7 +1186,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
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1542,10 +1596,11 @@ pub fn create_test_timeline(
|
||||
tenant: &crate::tenant::Tenant,
|
||||
timeline_id: utils::id::TimelineId,
|
||||
pg_version: u32,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<std::sync::Arc<Timeline>> {
|
||||
let tline = tenant
|
||||
.create_empty_timeline(timeline_id, Lsn(8), pg_version)?
|
||||
.initialize()?;
|
||||
.create_empty_timeline(timeline_id, Lsn(8), pg_version, ctx)?
|
||||
.initialize(ctx)?;
|
||||
let mut m = tline.begin_modification(Lsn(8));
|
||||
m.init_empty()?;
|
||||
m.commit()?;
|
||||
|
||||
@@ -171,6 +171,9 @@ task_local! {
|
||||
///
|
||||
#[derive(Debug, PartialEq, Eq, Clone, Copy)]
|
||||
pub enum TaskKind {
|
||||
// Pageserver startup, i.e., `main`
|
||||
Startup,
|
||||
|
||||
// libpq listener task. It just accepts connection and spawns a
|
||||
// PageRequestHandler task for each connection.
|
||||
LibpqEndpointListener,
|
||||
@@ -194,19 +197,26 @@ pub enum TaskKind {
|
||||
/// The `Connection` object is responsible for speaking the wire protocol.
|
||||
///
|
||||
/// Walreceiver uses its own abstraction called `TaskHandle` to represent the activity of establishing and handling a connection.
|
||||
/// That abstraction doesn't use `task_mgr` and hence, has no `TaskKind`.
|
||||
/// That abstraction doesn't use `task_mgr`.
|
||||
/// The [`WalReceiverManager`] task ensures that this `TaskHandle` task does not outlive the [`WalReceiverManager`] task.
|
||||
/// For the `RequestContext` that we hand to the TaskHandle, we use the [`WalReceiverConnectionHandler`] task kind.
|
||||
///
|
||||
/// Once the connection is established, the `TaskHandle` task creates a
|
||||
/// [`WalReceiverConnection`] task_mgr task that is responsible for polling
|
||||
/// [`WalReceiverConnectionPoller`] task_mgr task that is responsible for polling
|
||||
/// the `Connection` object.
|
||||
/// A `CancellationToken` created by the `TaskHandle` task ensures
|
||||
/// that the [`WalReceiverConnection`] task will cancel soon after as the `TaskHandle` is dropped.
|
||||
/// that the [`WalReceiverConnectionPoller`] task will cancel soon after as the `TaskHandle` is dropped.
|
||||
WalReceiverManager,
|
||||
|
||||
/// The task that polls the `tokio-postgres::Connection` object.
|
||||
/// The `TaskHandle` task that executes [`walreceiver_connection::handle_walreceiver_connection`].
|
||||
/// Not a `task_mgr` task, but we use this `TaskKind` for its `RequestContext`.
|
||||
/// See the comment on [`WalReceiverManager`].
|
||||
WalReceiverConnection,
|
||||
WalReceiverConnectionHandler,
|
||||
|
||||
/// The task that polls the `tokio-postgres::Connection` object.
|
||||
/// Spawned by task [`WalReceiverConnectionHandler`].
|
||||
/// See the comment on [`WalReceiverManager`].
|
||||
WalReceiverConnectionPoller,
|
||||
|
||||
// Garbage collection worker. One per tenant
|
||||
GarbageCollector,
|
||||
@@ -217,6 +227,8 @@ pub enum TaskKind {
|
||||
// Initial logical size calculation
|
||||
InitialLogicalSizeCalculation,
|
||||
|
||||
OndemandLogicalSizeCalculation,
|
||||
|
||||
// Task that flushes frozen in-memory layers to disk
|
||||
LayerFlushTask,
|
||||
|
||||
@@ -239,6 +251,12 @@ pub enum TaskKind {
|
||||
DownloadAllRemoteLayers,
|
||||
// Task that calculates synthetis size for all active tenants
|
||||
CalculateSyntheticSize,
|
||||
|
||||
// A request that comes in via the pageserver HTTP API.
|
||||
MgmtRequest,
|
||||
|
||||
#[cfg(test)]
|
||||
UnitTest,
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
|
||||
@@ -48,6 +48,7 @@ use std::time::{Duration, Instant};
|
||||
use self::metadata::TimelineMetadata;
|
||||
use self::remote_timeline_client::RemoteTimelineClient;
|
||||
use crate::config::PageServerConf;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::import_datadir;
|
||||
use crate::is_uninit_mark;
|
||||
use crate::metrics::{remove_tenant_metrics, STORAGE_TIME};
|
||||
@@ -174,7 +175,7 @@ impl UninitializedTimeline<'_> {
|
||||
///
|
||||
/// The new timeline is initialized in Active state, and its background jobs are
|
||||
/// started
|
||||
pub fn initialize(self) -> anyhow::Result<Arc<Timeline>> {
|
||||
pub fn initialize(self, _ctx: &RequestContext) -> anyhow::Result<Arc<Timeline>> {
|
||||
let mut timelines = self.owning_tenant.timelines.lock().unwrap();
|
||||
self.initialize_with_lock(&mut timelines, true, true)
|
||||
}
|
||||
@@ -239,11 +240,12 @@ impl UninitializedTimeline<'_> {
|
||||
self,
|
||||
copyin_stream: &mut (impl Stream<Item = io::Result<Bytes>> + Sync + Send + Unpin),
|
||||
base_lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Arc<Timeline>> {
|
||||
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")?;
|
||||
|
||||
@@ -261,9 +263,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<Timeline>> {
|
||||
@@ -449,6 +449,7 @@ impl Tenant {
|
||||
///
|
||||
/// If the operation fails, the timeline is left in the tenant's hash map in Broken state. On success,
|
||||
/// it is marked as Active.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
async fn timeline_init_and_sync(
|
||||
&self,
|
||||
timeline_id: TimelineId,
|
||||
@@ -457,6 +458,7 @@ impl Tenant {
|
||||
local_metadata: Option<TimelineMetadata>,
|
||||
ancestor: Option<Arc<Timeline>>,
|
||||
first_save: bool,
|
||||
_ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
let tenant_id = self.tenant_id;
|
||||
|
||||
@@ -572,6 +574,7 @@ impl Tenant {
|
||||
conf: &'static PageServerConf,
|
||||
tenant_id: TenantId,
|
||||
remote_storage: GenericRemoteStorage,
|
||||
ctx: &RequestContext,
|
||||
) -> Arc<Tenant> {
|
||||
// XXX: Attach should provide the config, especially during tenant migration.
|
||||
// See https://github.com/neondatabase/neon/issues/1555
|
||||
@@ -590,6 +593,7 @@ impl Tenant {
|
||||
// Do all the hard work in the background
|
||||
let tenant_clone = Arc::clone(&tenant);
|
||||
|
||||
let ctx = ctx.detached_child(TaskKind::Attach, DownloadBehavior::Warn);
|
||||
task_mgr::spawn(
|
||||
&tokio::runtime::Handle::current(),
|
||||
TaskKind::Attach,
|
||||
@@ -598,7 +602,7 @@ impl Tenant {
|
||||
"attach tenant",
|
||||
false,
|
||||
async move {
|
||||
match tenant_clone.attach().await {
|
||||
match tenant_clone.attach(ctx).await {
|
||||
Ok(_) => {}
|
||||
Err(e) => {
|
||||
tenant_clone.set_broken(&e.to_string());
|
||||
@@ -614,8 +618,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<Tenant>) -> anyhow::Result<()> {
|
||||
#[instrument(skip(self, ctx), fields(tenant_id=%self.tenant_id))]
|
||||
async fn attach(self: &Arc<Tenant>, ctx: RequestContext) -> 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.
|
||||
@@ -715,6 +719,7 @@ impl Tenant {
|
||||
index_parts.remove(&timeline_id).unwrap(),
|
||||
remote_metadata,
|
||||
remote_clients.remove(&timeline_id).unwrap(),
|
||||
&ctx,
|
||||
)
|
||||
.await
|
||||
.with_context(|| {
|
||||
@@ -764,6 +769,7 @@ impl Tenant {
|
||||
index_part: IndexPart,
|
||||
remote_metadata: TimelineMetadata,
|
||||
remote_client: RemoteTimelineClient,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
info!("downloading index file for timeline {}", timeline_id);
|
||||
tokio::fs::create_dir_all(self.conf.timeline_path(&timeline_id, &self.tenant_id))
|
||||
@@ -798,6 +804,7 @@ impl Tenant {
|
||||
local_metadata,
|
||||
ancestor,
|
||||
true,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
}
|
||||
@@ -826,11 +833,12 @@ impl Tenant {
|
||||
/// If the loading fails for some reason, the Tenant will go into Broken
|
||||
/// state.
|
||||
///
|
||||
#[instrument(skip(conf, remote_storage), fields(tenant_id=%tenant_id))]
|
||||
#[instrument(skip(conf, remote_storage, ctx), fields(tenant_id=%tenant_id))]
|
||||
pub fn spawn_load(
|
||||
conf: &'static PageServerConf,
|
||||
tenant_id: TenantId,
|
||||
remote_storage: Option<GenericRemoteStorage>,
|
||||
ctx: &RequestContext,
|
||||
) -> Arc<Tenant> {
|
||||
let tenant_conf = match Self::load_tenant_config(conf, tenant_id) {
|
||||
Ok(conf) => conf,
|
||||
@@ -854,6 +862,7 @@ impl Tenant {
|
||||
// Do all the hard work in a background task
|
||||
let tenant_clone = Arc::clone(&tenant);
|
||||
|
||||
let ctx = ctx.detached_child(TaskKind::InitialLoad, DownloadBehavior::Warn);
|
||||
let _ = task_mgr::spawn(
|
||||
&tokio::runtime::Handle::current(),
|
||||
TaskKind::InitialLoad,
|
||||
@@ -862,7 +871,7 @@ impl Tenant {
|
||||
"initial tenant load",
|
||||
false,
|
||||
async move {
|
||||
match tenant_clone.load().await {
|
||||
match tenant_clone.load(&ctx).await {
|
||||
Ok(()) => {}
|
||||
Err(err) => {
|
||||
tenant_clone.set_broken(&err.to_string());
|
||||
@@ -883,8 +892,8 @@ 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<Tenant>) -> anyhow::Result<()> {
|
||||
#[instrument(skip(self, ctx), fields(tenant_id=%self.tenant_id))]
|
||||
async fn load(self: &Arc<Tenant>, ctx: &RequestContext) -> anyhow::Result<()> {
|
||||
info!("loading tenant task");
|
||||
|
||||
utils::failpoint_sleep_millis_async!("before-loading-tenant");
|
||||
@@ -995,7 +1004,7 @@ 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, ctx)
|
||||
.await
|
||||
.with_context(|| format!("load local timeline {timeline_id}"))?;
|
||||
}
|
||||
@@ -1012,11 +1021,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, ctx), fields(timeline_id=%timeline_id))]
|
||||
async fn load_local_timeline(
|
||||
&self,
|
||||
timeline_id: TimelineId,
|
||||
local_metadata: TimelineMetadata,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
let ancestor = if let Some(ancestor_timeline_id) = local_metadata.ancestor_timeline() {
|
||||
let ancestor_timeline = self.get_timeline(ancestor_timeline_id, false)
|
||||
@@ -1060,6 +1070,7 @@ impl Tenant {
|
||||
Some(local_metadata),
|
||||
ancestor,
|
||||
false,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
}
|
||||
@@ -1111,6 +1122,7 @@ impl Tenant {
|
||||
new_timeline_id: TimelineId,
|
||||
initdb_lsn: Lsn,
|
||||
pg_version: u32,
|
||||
_ctx: &RequestContext,
|
||||
) -> anyhow::Result<UninitializedTimeline> {
|
||||
anyhow::ensure!(
|
||||
self.is_active(),
|
||||
@@ -1152,6 +1164,7 @@ impl Tenant {
|
||||
ancestor_timeline_id: Option<TimelineId>,
|
||||
mut ancestor_start_lsn: Option<Lsn>,
|
||||
pg_version: u32,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Option<Arc<Timeline>>> {
|
||||
anyhow::ensure!(
|
||||
self.is_active(),
|
||||
@@ -1189,13 +1202,21 @@ 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, ctx).await?;
|
||||
}
|
||||
|
||||
self.branch_timeline(ancestor_timeline_id, new_timeline_id, ancestor_start_lsn)
|
||||
self.branch_timeline(
|
||||
ancestor_timeline_id,
|
||||
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))
|
||||
@@ -1219,6 +1240,7 @@ impl Tenant {
|
||||
target_timeline_id: Option<TimelineId>,
|
||||
horizon: u64,
|
||||
pitr: Duration,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<GcResult> {
|
||||
anyhow::ensure!(
|
||||
self.is_active(),
|
||||
@@ -1233,7 +1255,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
|
||||
}
|
||||
}
|
||||
@@ -1242,7 +1264,7 @@ 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, ctx: &RequestContext) -> anyhow::Result<()> {
|
||||
anyhow::ensure!(
|
||||
self.is_active(),
|
||||
"Cannot run compaction iteration on inactive tenant"
|
||||
@@ -1264,7 +1286,7 @@ impl Tenant {
|
||||
|
||||
for (timeline_id, timeline) in &timelines_to_compact {
|
||||
timeline
|
||||
.compact()
|
||||
.compact(ctx)
|
||||
.instrument(info_span!("compact_timeline", timeline = %timeline_id))
|
||||
.await?;
|
||||
}
|
||||
@@ -1297,7 +1319,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: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// Transition the timeline into TimelineState::Stopping.
|
||||
// This should prevent new operations from starting.
|
||||
let timeline = {
|
||||
@@ -1869,12 +1895,13 @@ impl Tenant {
|
||||
target_timeline_id: Option<TimelineId>,
|
||||
horizon: u64,
|
||||
pitr: Duration,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<GcResult> {
|
||||
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, ctx)
|
||||
.await?;
|
||||
|
||||
utils::failpoint_sleep_millis_async!("gc_iteration_internal_after_getting_gc_timelines");
|
||||
@@ -1915,7 +1942,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<Vec<Arc<Timeline>>> {
|
||||
pub async fn refresh_gc_info(
|
||||
&self,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Vec<Arc<Timeline>>> {
|
||||
// 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.
|
||||
@@ -1925,7 +1955,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
|
||||
}
|
||||
|
||||
@@ -1934,6 +1964,7 @@ impl Tenant {
|
||||
target_timeline_id: Option<TimelineId>,
|
||||
horizon: u64,
|
||||
pitr: Duration,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Vec<Arc<Timeline>>> {
|
||||
// grab mutex to prevent new timelines from being created here.
|
||||
let gc_cs = self.gc_cs.lock().await;
|
||||
@@ -2005,7 +2036,9 @@ impl Tenant {
|
||||
))
|
||||
.map(|&x| x.1)
|
||||
.collect();
|
||||
timeline.update_gc_info(branchpoints, cutoff, pitr).await?;
|
||||
timeline
|
||||
.update_gc_info(branchpoints, cutoff, pitr, ctx)
|
||||
.await?;
|
||||
|
||||
gc_timelines.push(timeline);
|
||||
}
|
||||
@@ -2020,6 +2053,7 @@ impl Tenant {
|
||||
src: TimelineId,
|
||||
dst: TimelineId,
|
||||
start_lsn: Option<Lsn>,
|
||||
_ctx: &RequestContext,
|
||||
) -> anyhow::Result<Arc<Timeline>> {
|
||||
// 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
|
||||
@@ -2120,6 +2154,7 @@ impl Tenant {
|
||||
&self,
|
||||
timeline_id: TimelineId,
|
||||
pg_version: u32,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Arc<Timeline>> {
|
||||
let timeline_uninit_mark = {
|
||||
let timelines = self.timelines.lock().unwrap();
|
||||
@@ -2179,6 +2214,7 @@ impl Tenant {
|
||||
unfinished_timeline,
|
||||
pgdata_path,
|
||||
pgdata_lsn,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
.with_context(|| {
|
||||
@@ -2350,7 +2386,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<size::ModelInputs> {
|
||||
pub async fn gather_size_inputs(
|
||||
&self,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<size::ModelInputs> {
|
||||
let logical_sizes_at_once = self
|
||||
.conf
|
||||
.concurrent_tenant_size_logical_size_queries
|
||||
@@ -2362,15 +2401,15 @@ 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
|
||||
}
|
||||
|
||||
/// Calculate synthetic tenant size
|
||||
/// This is periodically called by background worker.
|
||||
/// result is cached in tenant struct
|
||||
#[instrument(skip_all, fields(tenant_id=%self.tenant_id))]
|
||||
pub async fn calculate_synthetic_size(&self) -> anyhow::Result<u64> {
|
||||
let inputs = self.gather_size_inputs().await?;
|
||||
pub async fn calculate_synthetic_size(&self, ctx: &RequestContext) -> anyhow::Result<u64> {
|
||||
let inputs = self.gather_size_inputs(ctx).await?;
|
||||
|
||||
let size = inputs.calculate()?;
|
||||
|
||||
@@ -2741,11 +2780,17 @@ pub mod harness {
|
||||
})
|
||||
}
|
||||
|
||||
pub async fn load(&self) -> Arc<Tenant> {
|
||||
self.try_load().await.expect("failed to load test tenant")
|
||||
pub async fn load(&self) -> (Arc<Tenant>, RequestContext) {
|
||||
let ctx = RequestContext::new(TaskKind::UnitTest, DownloadBehavior::Error);
|
||||
(
|
||||
self.try_load(&ctx)
|
||||
.await
|
||||
.expect("failed to load test tenant"),
|
||||
ctx,
|
||||
)
|
||||
}
|
||||
|
||||
pub async fn try_load(&self) -> anyhow::Result<Arc<Tenant>> {
|
||||
pub async fn try_load(&self, ctx: &RequestContext) -> anyhow::Result<Arc<Tenant>> {
|
||||
let walredo_mgr = Arc::new(TestRedoManager);
|
||||
|
||||
let tenant = Arc::new(Tenant::new(
|
||||
@@ -2773,8 +2818,7 @@ pub mod harness {
|
||||
timelines_to_load.insert(timeline_id, timeline_metadata);
|
||||
}
|
||||
// FIXME starts background jobs
|
||||
tenant.load().await?;
|
||||
|
||||
tenant.load(ctx).await?;
|
||||
Ok(tenant)
|
||||
}
|
||||
|
||||
@@ -2831,10 +2875,9 @@ 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) = TenantHarness::create("test_basic")?.load().await;
|
||||
let tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
|
||||
let writer = tline.writer();
|
||||
writer.put(*TEST_KEY, Lsn(0x10), &Value::Image(TEST_IMG("foo at 0x10")))?;
|
||||
@@ -2847,15 +2890,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")
|
||||
);
|
||||
|
||||
@@ -2864,14 +2907,14 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn no_duplicate_timelines() -> anyhow::Result<()> {
|
||||
let tenant = TenantHarness::create("no_duplicate_timelines")?
|
||||
let (tenant, ctx) = TenantHarness::create("no_duplicate_timelines")?
|
||||
.load()
|
||||
.await;
|
||||
let _ = tenant
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)?
|
||||
.initialize()?;
|
||||
let timeline =
|
||||
tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let _ = timeline.initialize(&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, &ctx) {
|
||||
Ok(_) => panic!("duplicate timeline creation should fail"),
|
||||
Err(e) => assert_eq!(
|
||||
e.to_string(),
|
||||
@@ -2897,13 +2940,13 @@ 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, ctx) = TenantHarness::create("test_branch")?.load().await;
|
||||
let tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
let writer = tline.writer();
|
||||
|
||||
#[allow(non_snake_case)]
|
||||
let TEST_KEY_A: Key = Key::from_hex("112222222233333333444444445500000001").unwrap();
|
||||
#[allow(non_snake_case)]
|
||||
@@ -2923,7 +2966,7 @@ mod tests {
|
||||
|
||||
// Branch the history, modify relation differently on the new timeline
|
||||
tenant
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x30)))
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x30)), &ctx)
|
||||
.await?;
|
||||
let newtline = tenant
|
||||
.get_timeline(NEW_TIMELINE_ID, true)
|
||||
@@ -2934,15 +2977,15 @@ mod tests {
|
||||
|
||||
// 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), &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), &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), &ctx).await?)?,
|
||||
"foobar at 0x20"
|
||||
);
|
||||
|
||||
@@ -2994,13 +3037,12 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_prohibit_branch_creation_on_garbage_collected_data() -> anyhow::Result<()> {
|
||||
let tenant =
|
||||
let (tenant, ctx) =
|
||||
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 tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&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
|
||||
@@ -3008,12 +3050,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, &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(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x25)), &ctx)
|
||||
.await
|
||||
{
|
||||
Ok(_) => panic!("branching should have failed"),
|
||||
@@ -3032,16 +3074,16 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_prohibit_branch_creation_on_pre_initdb_lsn() -> anyhow::Result<()> {
|
||||
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) =
|
||||
TenantHarness::create("test_prohibit_branch_creation_on_pre_initdb_lsn")?
|
||||
.load()
|
||||
.await;
|
||||
let tline =
|
||||
tenant.create_empty_timeline(TIMELINE_ID, Lsn(0x50), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let _tline = tline.initialize(&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(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x25)), &ctx)
|
||||
.await
|
||||
{
|
||||
Ok(_) => panic!("branching should have failed"),
|
||||
@@ -3083,40 +3125,40 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_retain_data_in_parent_which_is_needed_for_child() -> anyhow::Result<()> {
|
||||
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) =
|
||||
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, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
make_some_layers(tline.as_ref(), Lsn(0x20)).await?;
|
||||
|
||||
tenant
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40)))
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40)), &ctx)
|
||||
.await?;
|
||||
let newtline = tenant
|
||||
.get_timeline(NEW_TIMELINE_ID, true)
|
||||
.expect("Should have a local timeline");
|
||||
// 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, &ctx)
|
||||
.await?;
|
||||
assert!(newtline.get(*TEST_KEY, Lsn(0x25)).await.is_ok());
|
||||
assert!(newtline.get(*TEST_KEY, Lsn(0x25), &ctx).await.is_ok());
|
||||
|
||||
Ok(())
|
||||
}
|
||||
#[tokio::test]
|
||||
async fn test_parent_keeps_data_forever_after_branching() -> anyhow::Result<()> {
|
||||
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) =
|
||||
TenantHarness::create("test_parent_keeps_data_forever_after_branching")?
|
||||
.load()
|
||||
.await;
|
||||
let tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
make_some_layers(tline.as_ref(), Lsn(0x20)).await?;
|
||||
|
||||
tenant
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40)))
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40)), &ctx)
|
||||
.await?;
|
||||
let newtline = tenant
|
||||
.get_timeline(NEW_TIMELINE_ID, true)
|
||||
@@ -3126,12 +3168,12 @@ mod tests {
|
||||
|
||||
// run gc on parent
|
||||
tenant
|
||||
.gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO)
|
||||
.gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO, &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), &ctx).await?,
|
||||
TEST_IMG(&format!("foo at {}", Lsn(0x40)))
|
||||
);
|
||||
|
||||
@@ -3143,14 +3185,14 @@ mod tests {
|
||||
const TEST_NAME: &str = "timeline_load";
|
||||
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) = harness.load().await;
|
||||
let tline =
|
||||
tenant.create_empty_timeline(TIMELINE_ID, Lsn(0x8000), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
make_some_layers(tline.as_ref(), Lsn(0x8000)).await?;
|
||||
}
|
||||
|
||||
let tenant = harness.load().await;
|
||||
let (tenant, _ctx) = harness.load().await;
|
||||
tenant
|
||||
.get_timeline(TIMELINE_ID, true)
|
||||
.expect("cannot load timeline");
|
||||
@@ -3164,15 +3206,15 @@ mod tests {
|
||||
let harness = TenantHarness::create(TEST_NAME)?;
|
||||
// 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) = harness.load().await;
|
||||
let tline =
|
||||
tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
|
||||
make_some_layers(tline.as_ref(), Lsn(0x20)).await?;
|
||||
|
||||
tenant
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40)))
|
||||
.branch_timeline(TIMELINE_ID, NEW_TIMELINE_ID, Some(Lsn(0x40)), &ctx)
|
||||
.await?;
|
||||
|
||||
let newtline = tenant
|
||||
@@ -3183,7 +3225,7 @@ mod tests {
|
||||
}
|
||||
|
||||
// check that both of them are initially unloaded
|
||||
let tenant = harness.load().await;
|
||||
let (tenant, _ctx) = harness.load().await;
|
||||
|
||||
// check that both, child and ancestor are loaded
|
||||
let _child_tline = tenant
|
||||
@@ -3201,11 +3243,11 @@ 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;
|
||||
let (tenant, ctx) = harness.load().await;
|
||||
|
||||
tenant
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)?
|
||||
.initialize()?;
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?
|
||||
.initialize(&ctx)?;
|
||||
drop(tenant);
|
||||
|
||||
let metadata_path = harness.timeline_path(&TIMELINE_ID).join(METADATA_FILE_NAME);
|
||||
@@ -3217,7 +3259,7 @@ mod tests {
|
||||
metadata_bytes[8] ^= 1;
|
||||
std::fs::write(metadata_path, metadata_bytes)?;
|
||||
|
||||
let err = harness.try_load().await.err().expect("should fail");
|
||||
let err = harness.try_load(&ctx).await.err().expect("should fail");
|
||||
assert!(err
|
||||
.to_string()
|
||||
.starts_with("Failed to parse metadata bytes from path"));
|
||||
@@ -3241,10 +3283,9 @@ 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) = TenantHarness::create("test_images")?.load().await;
|
||||
let tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
|
||||
let writer = tline.writer();
|
||||
writer.put(*TEST_KEY, Lsn(0x10), &Value::Image(TEST_IMG("foo at 0x10")))?;
|
||||
@@ -3252,7 +3293,7 @@ mod tests {
|
||||
drop(writer);
|
||||
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
|
||||
let writer = tline.writer();
|
||||
writer.put(*TEST_KEY, Lsn(0x20), &Value::Image(TEST_IMG("foo at 0x20")))?;
|
||||
@@ -3260,7 +3301,7 @@ mod tests {
|
||||
drop(writer);
|
||||
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
|
||||
let writer = tline.writer();
|
||||
writer.put(*TEST_KEY, Lsn(0x30), &Value::Image(TEST_IMG("foo at 0x30")))?;
|
||||
@@ -3268,7 +3309,7 @@ mod tests {
|
||||
drop(writer);
|
||||
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
|
||||
let writer = tline.writer();
|
||||
writer.put(*TEST_KEY, Lsn(0x40), &Value::Image(TEST_IMG("foo at 0x40")))?;
|
||||
@@ -3276,26 +3317,26 @@ mod tests {
|
||||
drop(writer);
|
||||
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
|
||||
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")
|
||||
);
|
||||
assert_eq!(
|
||||
tline.get(*TEST_KEY, Lsn(0x30)).await?,
|
||||
tline.get(*TEST_KEY, Lsn(0x30), &ctx).await?,
|
||||
TEST_IMG("foo at 0x30")
|
||||
);
|
||||
assert_eq!(
|
||||
tline.get(*TEST_KEY, Lsn(0x40)).await?,
|
||||
tline.get(*TEST_KEY, Lsn(0x40), &ctx).await?,
|
||||
TEST_IMG("foo at 0x40")
|
||||
);
|
||||
|
||||
@@ -3308,10 +3349,9 @@ 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) = TenantHarness::create("test_bulk_insert")?.load().await;
|
||||
let tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
|
||||
let mut lsn = Lsn(0x10);
|
||||
|
||||
@@ -3340,10 +3380,10 @@ 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, &ctx)
|
||||
.await?;
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
tline.gc().await?;
|
||||
}
|
||||
|
||||
@@ -3352,10 +3392,9 @@ 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) = TenantHarness::create("test_random_updates")?.load().await;
|
||||
let tline = tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let tline = tline.initialize(&ctx)?;
|
||||
|
||||
const NUM_KEYS: usize = 1000;
|
||||
|
||||
@@ -3405,7 +3444,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, &ctx).await?,
|
||||
TEST_IMG(&format!("{} at {}", blknum, last_lsn))
|
||||
);
|
||||
}
|
||||
@@ -3413,10 +3452,10 @@ 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, &ctx)
|
||||
.await?;
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
tline.gc().await?;
|
||||
}
|
||||
|
||||
@@ -3425,12 +3464,12 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_traverse_branches() -> anyhow::Result<()> {
|
||||
let tenant = TenantHarness::create("test_traverse_branches")?
|
||||
let (tenant, ctx) = TenantHarness::create("test_traverse_branches")?
|
||||
.load()
|
||||
.await;
|
||||
let mut tline = tenant
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)?
|
||||
.initialize()?;
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?
|
||||
.initialize(&ctx)?;
|
||||
|
||||
const NUM_KEYS: usize = 1000;
|
||||
|
||||
@@ -3464,7 +3503,7 @@ mod tests {
|
||||
for _ in 0..50 {
|
||||
let new_tline_id = TimelineId::generate();
|
||||
tenant
|
||||
.branch_timeline(tline_id, new_tline_id, Some(lsn))
|
||||
.branch_timeline(tline_id, new_tline_id, Some(lsn), &ctx)
|
||||
.await?;
|
||||
tline = tenant
|
||||
.get_timeline(new_tline_id, true)
|
||||
@@ -3491,7 +3530,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, &ctx).await?,
|
||||
TEST_IMG(&format!("{} at {}", blknum, last_lsn))
|
||||
);
|
||||
}
|
||||
@@ -3499,10 +3538,10 @@ 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, &ctx)
|
||||
.await?;
|
||||
tline.freeze_and_flush().await?;
|
||||
tline.compact().await?;
|
||||
tline.compact(&ctx).await?;
|
||||
tline.gc().await?;
|
||||
}
|
||||
|
||||
@@ -3511,12 +3550,12 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_traverse_ancestors() -> anyhow::Result<()> {
|
||||
let tenant = TenantHarness::create("test_traverse_ancestors")?
|
||||
let (tenant, ctx) = TenantHarness::create("test_traverse_ancestors")?
|
||||
.load()
|
||||
.await;
|
||||
let mut tline = tenant
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION)?
|
||||
.initialize()?;
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?
|
||||
.initialize(&ctx)?;
|
||||
|
||||
const NUM_KEYS: usize = 100;
|
||||
const NUM_TLINES: usize = 50;
|
||||
@@ -3532,7 +3571,7 @@ mod tests {
|
||||
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_id, new_tline_id, Some(lsn), &ctx)
|
||||
.await?;
|
||||
tline = tenant
|
||||
.get_timeline(new_tline_id, true)
|
||||
@@ -3566,7 +3605,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, &ctx).await?,
|
||||
TEST_IMG(&format!("{idx} {blknum} at {lsn}"))
|
||||
);
|
||||
}
|
||||
|
||||
@@ -16,6 +16,7 @@ use remote_storage::GenericRemoteStorage;
|
||||
use utils::crashsafe;
|
||||
|
||||
use crate::config::PageServerConf;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::task_mgr::{self, TaskKind};
|
||||
use crate::tenant::config::TenantConfOpt;
|
||||
use crate::tenant::{Tenant, TenantState};
|
||||
@@ -43,6 +44,8 @@ pub async fn init_tenant_mgr(
|
||||
.await
|
||||
.with_context(|| format!("Failed to list tenants dir {tenants_dir:?}"))?;
|
||||
|
||||
let ctx = RequestContext::todo_child(TaskKind::Startup, DownloadBehavior::Warn);
|
||||
|
||||
loop {
|
||||
match dir_entries.next_entry().await {
|
||||
Ok(None) => break,
|
||||
@@ -86,6 +89,7 @@ pub async fn init_tenant_mgr(
|
||||
conf,
|
||||
&tenant_dir_path,
|
||||
remote_storage.clone(),
|
||||
&ctx,
|
||||
) {
|
||||
Ok(tenant) => {
|
||||
TENANTS.write().await.insert(tenant.tenant_id(), tenant);
|
||||
@@ -116,6 +120,7 @@ pub fn schedule_local_tenant_processing(
|
||||
conf: &'static PageServerConf,
|
||||
tenant_path: &Path,
|
||||
remote_storage: Option<GenericRemoteStorage>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Arc<Tenant>> {
|
||||
anyhow::ensure!(
|
||||
tenant_path.is_dir(),
|
||||
@@ -150,7 +155,7 @@ pub fn schedule_local_tenant_processing(
|
||||
let tenant = if conf.tenant_attaching_mark_file_path(&tenant_id).exists() {
|
||||
info!("tenant {tenant_id} has attaching mark file, resuming its attach operation");
|
||||
if let Some(remote_storage) = remote_storage {
|
||||
Tenant::spawn_attach(conf, tenant_id, remote_storage)
|
||||
Tenant::spawn_attach(conf, tenant_id, remote_storage, ctx)
|
||||
} else {
|
||||
warn!("tenant {tenant_id} has attaching mark file, but pageserver has no remote storage configured");
|
||||
Tenant::create_broken_tenant(conf, tenant_id)
|
||||
@@ -158,7 +163,7 @@ pub fn schedule_local_tenant_processing(
|
||||
} else {
|
||||
info!("tenant {tenant_id} is assumed to be loadable, starting load operation");
|
||||
// Start loading the tenant into memory. It will initially be in Loading state.
|
||||
Tenant::spawn_load(conf, tenant_id, remote_storage)
|
||||
Tenant::spawn_load(conf, tenant_id, remote_storage, ctx)
|
||||
};
|
||||
Ok(tenant)
|
||||
}
|
||||
@@ -207,6 +212,7 @@ pub async fn create_tenant(
|
||||
tenant_conf: TenantConfOpt,
|
||||
tenant_id: TenantId,
|
||||
remote_storage: Option<GenericRemoteStorage>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<Option<Arc<Tenant>>> {
|
||||
match TENANTS.write().await.entry(tenant_id) {
|
||||
hash_map::Entry::Occupied(_) => {
|
||||
@@ -218,7 +224,7 @@ pub async fn create_tenant(
|
||||
// If this section ever becomes contentious, introduce a new `TenantState::Creating`.
|
||||
let tenant_directory = super::create_tenant_files(conf, tenant_conf, tenant_id)?;
|
||||
let created_tenant =
|
||||
schedule_local_tenant_processing(conf, &tenant_directory, remote_storage)?;
|
||||
schedule_local_tenant_processing(conf, &tenant_directory, remote_storage, ctx)?;
|
||||
let crated_tenant_id = created_tenant.tenant_id();
|
||||
anyhow::ensure!(
|
||||
tenant_id == crated_tenant_id,
|
||||
@@ -260,10 +266,14 @@ 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?;
|
||||
tenant.delete_timeline(timeline_id, ctx).await?;
|
||||
}
|
||||
Err(e) => anyhow::bail!("Cannot access tenant {tenant_id} in local tenant state: {e:?}"),
|
||||
}
|
||||
@@ -291,6 +301,7 @@ pub async fn load_tenant(
|
||||
conf: &'static PageServerConf,
|
||||
tenant_id: TenantId,
|
||||
remote_storage: Option<GenericRemoteStorage>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
run_if_no_tenant_in_memory(tenant_id, |vacant_entry| {
|
||||
let tenant_path = conf.tenant_path(&tenant_id);
|
||||
@@ -300,7 +311,7 @@ pub async fn load_tenant(
|
||||
.with_context(|| format!("Failed to remove tenant ignore mark {tenant_ignore_mark:?} during tenant loading"))?;
|
||||
}
|
||||
|
||||
let new_tenant = schedule_local_tenant_processing(conf, &tenant_path, remote_storage)
|
||||
let new_tenant = schedule_local_tenant_processing(conf, &tenant_path, remote_storage, ctx)
|
||||
.with_context(|| {
|
||||
format!("Failed to schedule tenant processing in path {tenant_path:?}")
|
||||
})?;
|
||||
@@ -349,6 +360,7 @@ pub async fn attach_tenant(
|
||||
conf: &'static PageServerConf,
|
||||
tenant_id: TenantId,
|
||||
remote_storage: GenericRemoteStorage,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
run_if_no_tenant_in_memory(tenant_id, |vacant_entry| {
|
||||
let tenant_path = conf.tenant_path(&tenant_id);
|
||||
@@ -357,7 +369,7 @@ pub async fn attach_tenant(
|
||||
"Cannot attach tenant {tenant_id}, local tenant directory already exists"
|
||||
);
|
||||
|
||||
let tenant = Tenant::spawn_attach(conf, tenant_id, remote_storage);
|
||||
let tenant = Tenant::spawn_attach(conf, tenant_id, remote_storage, ctx);
|
||||
vacant_entry.insert(tenant);
|
||||
|
||||
Ok(())
|
||||
@@ -449,9 +461,9 @@ pub async fn immediate_gc(
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
gc_req: TimelineGcRequest,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<tokio::sync::oneshot::Receiver<Result<GcResult, anyhow::Error>>, ApiError> {
|
||||
let guard = TENANTS.read().await;
|
||||
|
||||
let tenant = guard
|
||||
.get(&tenant_id)
|
||||
.map(Arc::clone)
|
||||
@@ -462,7 +474,8 @@ pub async fn immediate_gc(
|
||||
// Use tenant's pitr setting
|
||||
let pitr = tenant.get_pitr_interval();
|
||||
|
||||
// Run in task_mgr to avoid race with detach operation
|
||||
// Run in task_mgr to avoid race with tenant_detach operation
|
||||
let ctx = ctx.detached_child(TaskKind::GarbageCollector, DownloadBehavior::Download);
|
||||
let (task_done, wait_task_done) = tokio::sync::oneshot::channel();
|
||||
task_mgr::spawn(
|
||||
&tokio::runtime::Handle::current(),
|
||||
@@ -474,7 +487,7 @@ pub async fn immediate_gc(
|
||||
async move {
|
||||
fail::fail_point!("immediate_gc_task_pre");
|
||||
let result = tenant
|
||||
.gc_iteration(Some(timeline_id), gc_horizon, pitr)
|
||||
.gc_iteration(Some(timeline_id), gc_horizon, pitr, &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
|
||||
@@ -497,6 +510,7 @@ pub async fn immediate_gc(
|
||||
pub async fn immediate_compact(
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<tokio::sync::oneshot::Receiver<anyhow::Result<()>>, ApiError> {
|
||||
let guard = TENANTS.read().await;
|
||||
|
||||
@@ -510,7 +524,8 @@ pub async fn immediate_compact(
|
||||
.get_timeline(timeline_id, true)
|
||||
.map_err(ApiError::NotFound)?;
|
||||
|
||||
// Run in task_mgr to avoid race with detach operation
|
||||
// Run in task_mgr to avoid race with tenant_detach operation
|
||||
let ctx = ctx.detached_child(TaskKind::Compaction, DownloadBehavior::Download);
|
||||
let (task_done, wait_task_done) = tokio::sync::oneshot::channel();
|
||||
task_mgr::spawn(
|
||||
&tokio::runtime::Handle::current(),
|
||||
@@ -523,7 +538,7 @@ pub async fn immediate_compact(
|
||||
false,
|
||||
async move {
|
||||
let result = timeline
|
||||
.compact()
|
||||
.compact(&ctx)
|
||||
.instrument(
|
||||
info_span!("manual_compact", tenant = %tenant_id, timeline = %timeline_id),
|
||||
)
|
||||
|
||||
@@ -1010,7 +1010,10 @@ impl RemoteTimelineClient {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::tenant::harness::{TenantHarness, TIMELINE_ID};
|
||||
use crate::{
|
||||
tenant::harness::{TenantHarness, TIMELINE_ID},
|
||||
DEFAULT_PG_VERSION,
|
||||
};
|
||||
use remote_storage::{RemoteStorageConfig, RemoteStorageKind};
|
||||
use std::{collections::HashSet, path::Path};
|
||||
use utils::lsn::Lsn;
|
||||
@@ -1064,9 +1067,19 @@ mod tests {
|
||||
// Test scheduling
|
||||
#[test]
|
||||
fn upload_scheduling() -> anyhow::Result<()> {
|
||||
// Use a current-thread runtime in the test
|
||||
let runtime = Box::leak(Box::new(
|
||||
tokio::runtime::Builder::new_current_thread()
|
||||
.enable_all()
|
||||
.build()?,
|
||||
));
|
||||
let _entered = runtime.enter();
|
||||
|
||||
let harness = TenantHarness::create("upload_scheduling")?;
|
||||
let (tenant, ctx) = runtime.block_on(harness.load());
|
||||
let _timeline =
|
||||
tenant.create_empty_timeline(TIMELINE_ID, Lsn(0), DEFAULT_PG_VERSION, &ctx)?;
|
||||
let timeline_path = harness.timeline_path(&TIMELINE_ID);
|
||||
std::fs::create_dir_all(&timeline_path)?;
|
||||
|
||||
let remote_fs_dir = harness.conf.workdir.join("remote_fs");
|
||||
std::fs::create_dir_all(remote_fs_dir)?;
|
||||
@@ -1084,14 +1097,6 @@ mod tests {
|
||||
storage: RemoteStorageKind::LocalFs(remote_fs_dir.clone()),
|
||||
};
|
||||
|
||||
// Use a current-thread runtime in the test
|
||||
let runtime = Box::leak(Box::new(
|
||||
tokio::runtime::Builder::new_current_thread()
|
||||
.enable_all()
|
||||
.build()?,
|
||||
));
|
||||
let _entered = runtime.enter();
|
||||
|
||||
// Test outline:
|
||||
//
|
||||
// Schedule upload of a bunch of layers. Check that they are started immediately, not queued
|
||||
|
||||
@@ -6,6 +6,7 @@ use anyhow::Context;
|
||||
use tokio::sync::oneshot::error::RecvError;
|
||||
use tokio::sync::Semaphore;
|
||||
|
||||
use crate::context::RequestContext;
|
||||
use crate::pgdatadir_mapping::CalculateLogicalSizeError;
|
||||
|
||||
use super::Tenant;
|
||||
@@ -181,6 +182,7 @@ pub(super) async fn gather_inputs(
|
||||
tenant: &Tenant,
|
||||
limit: &Arc<Semaphore>,
|
||||
logical_size_cache: &mut HashMap<(TimelineId, Lsn), u64>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<ModelInputs> {
|
||||
// with joinset, on drop, all of the tasks will just be de-scheduled, which we can use to
|
||||
// our advantage with `?` error handling.
|
||||
@@ -188,7 +190,7 @@ pub(super) async fn gather_inputs(
|
||||
|
||||
// refresh is needed to update gc related pitr_cutoff and horizon_cutoff
|
||||
tenant
|
||||
.refresh_gc_info()
|
||||
.refresh_gc_info(ctx)
|
||||
.await
|
||||
.context("Failed to refresh gc_info before gathering inputs")?;
|
||||
|
||||
@@ -329,7 +331,13 @@ pub(super) async fn gather_inputs(
|
||||
} 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 = ctx.attached_child();
|
||||
joinset.spawn(calculate_logical_size(
|
||||
parallel_size_calcs,
|
||||
timeline,
|
||||
*lsn,
|
||||
ctx,
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -387,6 +395,7 @@ pub(super) async fn gather_inputs(
|
||||
parallel_size_calcs,
|
||||
timeline.clone(),
|
||||
lsn,
|
||||
ctx.attached_child(),
|
||||
));
|
||||
|
||||
if let Some(parent_id) = timeline.get_ancestor_timeline_id() {
|
||||
@@ -582,13 +591,14 @@ async fn calculate_logical_size(
|
||||
limit: Arc<tokio::sync::Semaphore>,
|
||||
timeline: Arc<crate::tenant::Timeline>,
|
||||
lsn: utils::lsn::Lsn,
|
||||
ctx: RequestContext,
|
||||
) -> Result<TimelineAtLsnSizeResult, RecvError> {
|
||||
let _permit = tokio::sync::Semaphore::acquire_owned(limit)
|
||||
.await
|
||||
.expect("global semaphore should not had been closed");
|
||||
|
||||
let size_res = timeline
|
||||
.spawn_ondemand_logical_size_calculation(lsn)
|
||||
.spawn_ondemand_logical_size_calculation(lsn, ctx)
|
||||
.instrument(info_span!("spawn_ondemand_logical_size_calculation"))
|
||||
.await?;
|
||||
Ok(TimelineAtLsnSizeResult(timeline, lsn, size_res))
|
||||
|
||||
@@ -5,6 +5,7 @@ use std::ops::ControlFlow;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::metrics::TENANT_TASK_EVENTS;
|
||||
use crate::task_mgr;
|
||||
use crate::task_mgr::{TaskKind, BACKGROUND_RUNTIME};
|
||||
@@ -52,19 +53,20 @@ async fn compaction_loop(tenant_id: TenantId) {
|
||||
info!("starting");
|
||||
TENANT_TASK_EVENTS.with_label_values(&["start"]).inc();
|
||||
async {
|
||||
let ctx = RequestContext::todo_child(TaskKind::Compaction, DownloadBehavior::Download);
|
||||
loop {
|
||||
trace!("waking up");
|
||||
|
||||
let tenant = tokio::select! {
|
||||
_ = task_mgr::shutdown_watcher() => {
|
||||
info!("received cancellation request");
|
||||
return;
|
||||
return;
|
||||
},
|
||||
tenant_wait_result = wait_for_active_tenant(tenant_id, wait_duration) => match tenant_wait_result {
|
||||
ControlFlow::Break(()) => return,
|
||||
ControlFlow::Continue(tenant) => tenant,
|
||||
},
|
||||
};
|
||||
};
|
||||
|
||||
let mut sleep_duration = tenant.get_compaction_period();
|
||||
if sleep_duration == Duration::ZERO {
|
||||
@@ -73,7 +75,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(&ctx).await {
|
||||
sleep_duration = wait_duration;
|
||||
error!("Compaction failed, retrying in {:?}: {e:?}", sleep_duration);
|
||||
}
|
||||
@@ -103,6 +105,9 @@ async fn gc_loop(tenant_id: TenantId) {
|
||||
info!("starting");
|
||||
TENANT_TASK_EVENTS.with_label_values(&["start"]).inc();
|
||||
async {
|
||||
// GC might require downloading, to find the cutoff LSN that corresponds to the
|
||||
// cutoff specified as time.
|
||||
let ctx = RequestContext::todo_child(TaskKind::GarbageCollector, DownloadBehavior::Download);
|
||||
loop {
|
||||
trace!("waking up");
|
||||
|
||||
@@ -127,7 +132,7 @@ 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(), &ctx).await
|
||||
{
|
||||
sleep_duration = wait_duration;
|
||||
error!("Gc failed, retrying in {:?}: {e:?}", sleep_duration);
|
||||
|
||||
@@ -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::{DownloadBehavior, RequestContext};
|
||||
use crate::tenant::remote_timeline_client::{self, index::LayerFileMetadata};
|
||||
use crate::tenant::storage_layer::{
|
||||
DeltaFileName, DeltaLayerWriter, ImageFileName, ImageLayerWriter, InMemoryLayer, LayerFileName,
|
||||
@@ -377,6 +378,12 @@ pub enum PageReconstructError {
|
||||
#[error(transparent)]
|
||||
Other(#[from] anyhow::Error), // source and Display delegate to anyhow::Error
|
||||
|
||||
/// The operation would require downloading a layer that is missing locally.
|
||||
NeedsDownload(Weak<Timeline>, Weak<RemoteLayer>),
|
||||
|
||||
/// The operation was cancelled
|
||||
Cancelled,
|
||||
|
||||
/// An error happened replaying WAL records
|
||||
#[error(transparent)]
|
||||
WalRedo(#[from] crate::walredo::WalRedoError),
|
||||
@@ -386,6 +393,19 @@ impl std::fmt::Debug for PageReconstructError {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> {
|
||||
match self {
|
||||
Self::Other(err) => err.fmt(f),
|
||||
Self::NeedsDownload(_tli, _layer) => write!(f, "needs download"),
|
||||
Self::Cancelled => write!(f, "cancelled"),
|
||||
Self::WalRedo(err) => err.fmt(f),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl std::fmt::Display for PageReconstructError {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> {
|
||||
match self {
|
||||
Self::Other(err) => err.fmt(f),
|
||||
Self::NeedsDownload(_tli, _layer) => write!(f, "needs download"),
|
||||
Self::Cancelled => write!(f, "cancelled"),
|
||||
Self::WalRedo(err) => err.fmt(f),
|
||||
}
|
||||
}
|
||||
@@ -422,11 +442,24 @@ 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<Bytes, PageReconstructError> {
|
||||
pub async fn get(
|
||||
&self,
|
||||
key: Key,
|
||||
lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<Bytes, PageReconstructError> {
|
||||
if !lsn.is_valid() {
|
||||
return Err(PageReconstructError::Other(anyhow::anyhow!("Invalid LSN")));
|
||||
}
|
||||
|
||||
// XXX: structured stats collection for layer eviction here.
|
||||
trace!(
|
||||
"get page request for {}@{} from task kind {:?}",
|
||||
key,
|
||||
lsn,
|
||||
ctx.task_kind()
|
||||
);
|
||||
|
||||
// Check the page cache. We will get back the most recent page with lsn <= `lsn`.
|
||||
// The cached image can be returned directly if there is no WAL between the cached image
|
||||
// and requested LSN. The cached image can also be used to reduce the amount of WAL needed
|
||||
@@ -450,7 +483,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,13 +546,25 @@ 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: &RequestContext, /* Prepare for use by cancellation */
|
||||
) -> 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
|
||||
// to a deadlock.
|
||||
anyhow::ensure!(
|
||||
task_mgr::current_task_kind() != Some(TaskKind::WalReceiverConnection),
|
||||
task_mgr::current_task_kind() != Some(TaskKind::WalReceiverManager),
|
||||
"wait_lsn cannot be called in WAL receiver"
|
||||
);
|
||||
anyhow::ensure!(
|
||||
task_mgr::current_task_kind() != Some(TaskKind::WalReceiverConnectionHandler),
|
||||
"wait_lsn cannot be called in WAL receiver"
|
||||
);
|
||||
anyhow::ensure!(
|
||||
task_mgr::current_task_kind() != Some(TaskKind::WalReceiverConnectionPoller),
|
||||
"wait_lsn cannot be called in WAL receiver"
|
||||
);
|
||||
|
||||
@@ -558,7 +603,7 @@ impl Timeline {
|
||||
self.flush_frozen_layers_and_wait().await
|
||||
}
|
||||
|
||||
pub async fn compact(&self) -> anyhow::Result<()> {
|
||||
pub async fn compact(&self, ctx: &RequestContext) -> 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 +661,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)?;
|
||||
@@ -673,7 +720,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<Self>) -> anyhow::Result<(u64, bool)> {
|
||||
pub fn get_current_logical_size(
|
||||
self: &Arc<Self>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<(u64, bool)> {
|
||||
let current_size = self.current_logical_size.current_size()?;
|
||||
debug!("Current size: {current_size:?}");
|
||||
|
||||
@@ -683,7 +733,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))
|
||||
@@ -917,22 +967,25 @@ impl Timeline {
|
||||
|
||||
let layer_flush_start_rx = self.layer_flush_start_tx.subscribe();
|
||||
let self_clone = Arc::clone(self);
|
||||
|
||||
info!("spawning flush loop");
|
||||
task_mgr::spawn(
|
||||
task_mgr::BACKGROUND_RUNTIME.handle(),
|
||||
task_mgr::TaskKind::LayerFlushTask,
|
||||
Some(self.tenant_id),
|
||||
Some(self.timeline_id),
|
||||
"layer flush task",
|
||||
false,
|
||||
async move {
|
||||
self_clone.flush_loop(layer_flush_start_rx).await;
|
||||
let mut flush_loop_state = self_clone.flush_loop_state.lock().unwrap();
|
||||
assert_eq!(*flush_loop_state, FlushLoopState::Running);
|
||||
*flush_loop_state = FlushLoopState::Exited;
|
||||
Ok(()) }
|
||||
.instrument(info_span!(parent: None, "layer flush task", tenant = %self.tenant_id, timeline = %self.timeline_id))
|
||||
);
|
||||
task_mgr::BACKGROUND_RUNTIME.handle(),
|
||||
task_mgr::TaskKind::LayerFlushTask,
|
||||
Some(self.tenant_id),
|
||||
Some(self.timeline_id),
|
||||
"layer flush task",
|
||||
false,
|
||||
async move {
|
||||
let background_ctx = RequestContext::todo_child(TaskKind::LayerFlushTask, DownloadBehavior::Error);
|
||||
self_clone.flush_loop(layer_flush_start_rx, &background_ctx).await;
|
||||
let mut flush_loop_state = self_clone.flush_loop_state.lock().unwrap();
|
||||
assert_eq!(*flush_loop_state, FlushLoopState::Running);
|
||||
*flush_loop_state = FlushLoopState::Exited;
|
||||
Ok(())
|
||||
}
|
||||
.instrument(info_span!(parent: None, "layer flush task", tenant = %self.tenant_id, timeline = %self.timeline_id))
|
||||
);
|
||||
|
||||
*flush_loop_state = FlushLoopState::Running;
|
||||
}
|
||||
@@ -963,12 +1016,16 @@ impl Timeline {
|
||||
.unwrap_or(self.conf.default_tenant_conf.max_lsn_wal_lag);
|
||||
drop(tenant_conf_guard);
|
||||
let self_clone = Arc::clone(self);
|
||||
let background_ctx =
|
||||
// XXX: this is a detached_child. Plumb through the ctx from call sites.
|
||||
RequestContext::todo_child(TaskKind::WalReceiverManager, DownloadBehavior::Error);
|
||||
spawn_connection_manager_task(
|
||||
self_clone,
|
||||
walreceiver_connect_timeout,
|
||||
lagging_wal_timeout,
|
||||
max_lsn_wal_lag,
|
||||
crate::config::SAFEKEEPER_AUTH_TOKEN.get().cloned(),
|
||||
background_ctx,
|
||||
);
|
||||
}
|
||||
|
||||
@@ -1296,7 +1353,7 @@ impl Timeline {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn try_spawn_size_init_task(self: &Arc<Self>, init_lsn: Lsn) {
|
||||
fn try_spawn_size_init_task(self: &Arc<Self>, init_lsn: Lsn, ctx: &RequestContext) {
|
||||
let permit = match Arc::clone(&self.current_logical_size.initial_size_computation)
|
||||
.try_acquire_owned()
|
||||
{
|
||||
@@ -1312,8 +1369,18 @@ impl Timeline {
|
||||
.initial_logical_size
|
||||
.get()
|
||||
.is_none());
|
||||
|
||||
info!(
|
||||
"spawning logical size computation from context of task kind {:?}",
|
||||
ctx.task_kind()
|
||||
);
|
||||
// We need to start the computation task.
|
||||
// It gets a separate context since it will outlive the request that called this function.
|
||||
let self_clone = Arc::clone(self);
|
||||
let background_ctx = ctx.detached_child(
|
||||
TaskKind::InitialLogicalSizeCalculation,
|
||||
DownloadBehavior::Download,
|
||||
);
|
||||
task_mgr::spawn(
|
||||
task_mgr::BACKGROUND_RUNTIME.handle(),
|
||||
task_mgr::TaskKind::InitialLogicalSizeCalculation,
|
||||
@@ -1323,7 +1390,9 @@ impl Timeline {
|
||||
false,
|
||||
// NB: don't log errors here, task_mgr will do that.
|
||||
async move {
|
||||
let calculated_size = match self_clone.logical_size_calculation_task(init_lsn).await
|
||||
let calculated_size = match self_clone
|
||||
.logical_size_calculation_task(init_lsn, &background_ctx)
|
||||
.await
|
||||
{
|
||||
Ok(s) => s,
|
||||
Err(CalculateLogicalSizeError::Cancelled) => {
|
||||
@@ -1358,18 +1427,27 @@ impl Timeline {
|
||||
pub fn spawn_ondemand_logical_size_calculation(
|
||||
self: &Arc<Self>,
|
||||
lsn: Lsn,
|
||||
ctx: RequestContext,
|
||||
) -> oneshot::Receiver<Result<u64, CalculateLogicalSizeError>> {
|
||||
let (sender, receiver) = oneshot::channel();
|
||||
let self_clone = Arc::clone(self);
|
||||
// XXX if our caller loses interest, i.e., ctx is cancelled,
|
||||
// we should stop the size calculation work and return an error.
|
||||
// That would require restructuring this function's API to
|
||||
// return the result directly, instead of a Receiver for the result.
|
||||
let ctx = ctx.detached_child(
|
||||
TaskKind::OndemandLogicalSizeCalculation,
|
||||
DownloadBehavior::Download,
|
||||
);
|
||||
task_mgr::spawn(
|
||||
task_mgr::BACKGROUND_RUNTIME.handle(),
|
||||
task_mgr::TaskKind::InitialLogicalSizeCalculation,
|
||||
task_mgr::TaskKind::OndemandLogicalSizeCalculation,
|
||||
Some(self.tenant_id),
|
||||
Some(self.timeline_id),
|
||||
"ondemand logical size calculation",
|
||||
false,
|
||||
async move {
|
||||
let res = self_clone.logical_size_calculation_task(lsn).await;
|
||||
let res = self_clone.logical_size_calculation_task(lsn, &ctx).await;
|
||||
let _ = sender.send(res).ok();
|
||||
Ok(()) // Receiver is responsible for handling errors
|
||||
},
|
||||
@@ -1381,6 +1459,7 @@ impl Timeline {
|
||||
async fn logical_size_calculation_task(
|
||||
self: &Arc<Self>,
|
||||
init_lsn: Lsn,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<u64, CalculateLogicalSizeError> {
|
||||
let mut timeline_state_updates = self.subscribe_for_state_updates();
|
||||
let self_calculation = Arc::clone(self);
|
||||
@@ -1388,12 +1467,13 @@ impl Timeline {
|
||||
|
||||
let calculation = async {
|
||||
let cancel = cancel.child_token();
|
||||
let ctx = ctx.attached_child();
|
||||
tokio::task::spawn_blocking(move || {
|
||||
// Run in a separate thread since this can do a lot of
|
||||
// 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))
|
||||
h.block_on(self_calculation.calculate_logical_size(init_lsn, cancel, &ctx))
|
||||
})
|
||||
.await
|
||||
.context("Failed to spawn calculation result task")?
|
||||
@@ -1448,10 +1528,11 @@ impl Timeline {
|
||||
///
|
||||
/// NOTE: counted incrementally, includes ancestors. This can be a slow operation,
|
||||
/// especially if we need to download remote layers.
|
||||
async fn calculate_logical_size(
|
||||
pub async fn calculate_logical_size(
|
||||
&self,
|
||||
up_to_lsn: Lsn,
|
||||
cancel: CancellationToken,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<u64, CalculateLogicalSizeError> {
|
||||
info!(
|
||||
"Calculating logical size for timeline {} at {}",
|
||||
@@ -1494,7 +1575,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();
|
||||
@@ -1571,6 +1652,7 @@ impl Timeline {
|
||||
key: Key,
|
||||
request_lsn: Lsn,
|
||||
reconstruct_state: &mut ValueReconstructState,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<(), PageReconstructError> {
|
||||
// Start from the current timeline.
|
||||
let mut timeline_owned;
|
||||
@@ -1758,14 +1840,43 @@ impl Timeline {
|
||||
let remote_layer_as_persistent: Arc<dyn PersistentLayer> =
|
||||
Arc::clone(&remote_layer) as Arc<dyn PersistentLayer>;
|
||||
let id = remote_layer_as_persistent.traversal_id();
|
||||
info!("need remote layer {id}");
|
||||
info!(
|
||||
"need remote layer {} for task kind {:?}",
|
||||
id,
|
||||
ctx.task_kind()
|
||||
);
|
||||
|
||||
// The next layer doesn't exist locally. Need to download it.
|
||||
// (The control flow is a bit complicated here because we must drop the 'layers'
|
||||
// lock before awaiting on the Future.)
|
||||
info!("on-demand downloading remote layer {id}");
|
||||
timeline.download_remote_layer(remote_layer).await?;
|
||||
continue 'layer_map_search;
|
||||
match (
|
||||
ctx.download_behavior(),
|
||||
self.conf.ondemand_download_behavior_treat_error_as_warn,
|
||||
) {
|
||||
(DownloadBehavior::Download, _) => {
|
||||
info!(
|
||||
"on-demand downloading remote layer {id} for task kind {:?}",
|
||||
ctx.task_kind()
|
||||
);
|
||||
timeline.download_remote_layer(remote_layer).await?;
|
||||
continue 'layer_map_search;
|
||||
}
|
||||
(DownloadBehavior::Warn, _) | (DownloadBehavior::Error, true) => {
|
||||
warn!(
|
||||
"unexpectedly on-demand downloading remote layer {} for task kind {:?}",
|
||||
id,
|
||||
ctx.task_kind()
|
||||
);
|
||||
timeline.download_remote_layer(remote_layer).await?;
|
||||
continue 'layer_map_search;
|
||||
}
|
||||
(DownloadBehavior::Error, false) => {
|
||||
return Err(PageReconstructError::NeedsDownload(
|
||||
timeline.myself.clone(),
|
||||
Arc::downgrade(&remote_layer),
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1887,7 +1998,11 @@ impl Timeline {
|
||||
}
|
||||
|
||||
/// Layer flusher task's main loop.
|
||||
async fn flush_loop(&self, mut layer_flush_start_rx: tokio::sync::watch::Receiver<u64>) {
|
||||
async fn flush_loop(
|
||||
&self,
|
||||
mut layer_flush_start_rx: tokio::sync::watch::Receiver<u64>,
|
||||
ctx: &RequestContext,
|
||||
) {
|
||||
info!("started flush loop");
|
||||
loop {
|
||||
tokio::select! {
|
||||
@@ -1908,7 +2023,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);
|
||||
}
|
||||
@@ -1973,8 +2088,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<InMemoryLayer>) -> 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<InMemoryLayer>,
|
||||
ctx: &RequestContext,
|
||||
) -> 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
|
||||
@@ -1982,10 +2101,12 @@ impl Timeline {
|
||||
let lsn_range = frozen_layer.get_lsn_range();
|
||||
let layer_paths_to_upload =
|
||||
if lsn_range.start == self.initdb_lsn && lsn_range.end == Lsn(self.initdb_lsn.0 + 1) {
|
||||
// Note: The 'ctx' in use here has DownloadBehavior::Error. We should not
|
||||
// require downloading anything during initial import.
|
||||
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.
|
||||
@@ -2136,6 +2257,7 @@ impl Timeline {
|
||||
&self,
|
||||
lsn: Lsn,
|
||||
partition_size: u64,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<(KeyPartitioning, Lsn)> {
|
||||
{
|
||||
let partitioning_guard = self.partitioning.lock().unwrap();
|
||||
@@ -2146,7 +2268,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();
|
||||
@@ -2206,6 +2328,7 @@ impl Timeline {
|
||||
partitioning: &KeyPartitioning,
|
||||
lsn: Lsn,
|
||||
force: bool,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<HashMap<LayerFileName, LayerFileMetadata>, PageReconstructError> {
|
||||
let timer = self.metrics.create_images_time_histo.start_timer();
|
||||
let mut image_layers: Vec<ImageLayer> = Vec::new();
|
||||
@@ -2230,7 +2353,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
|
||||
@@ -2685,6 +2808,7 @@ impl Timeline {
|
||||
retain_lsns: Vec<Lsn>,
|
||||
cutoff_horizon: Lsn,
|
||||
pitr: Duration,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// First, calculate pitr_cutoff_timestamp and then convert it to LSN.
|
||||
//
|
||||
@@ -2697,7 +2821,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,
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -11,6 +11,7 @@
|
||||
|
||||
use std::{collections::HashMap, num::NonZeroU64, ops::ControlFlow, sync::Arc, time::Duration};
|
||||
|
||||
use crate::context::RequestContext;
|
||||
use crate::task_mgr::TaskKind;
|
||||
use crate::task_mgr::WALRECEIVER_RUNTIME;
|
||||
use crate::tenant::Timeline;
|
||||
@@ -46,6 +47,7 @@ pub fn spawn_connection_manager_task(
|
||||
lagging_wal_timeout: Duration,
|
||||
max_lsn_wal_lag: NonZeroU64,
|
||||
auth_token: Option<Arc<String>>,
|
||||
ctx: RequestContext,
|
||||
) {
|
||||
let mut broker_client = get_broker_client().clone();
|
||||
|
||||
@@ -78,6 +80,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 +104,7 @@ pub fn spawn_connection_manager_task(
|
||||
async fn connection_manager_loop_step(
|
||||
broker_client: &mut BrokerClientChannel,
|
||||
walreceiver_state: &mut WalreceiverState,
|
||||
ctx: &RequestContext,
|
||||
) -> ControlFlow<(), ()> {
|
||||
let mut timeline_state_updates = walreceiver_state.timeline.subscribe_for_state_updates();
|
||||
|
||||
@@ -236,6 +240,7 @@ async fn connection_manager_loop_step(
|
||||
.change_connection(
|
||||
new_candidate.safekeeper_id,
|
||||
new_candidate.wal_source_connconf,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
}
|
||||
@@ -401,12 +406,17 @@ impl WalreceiverState {
|
||||
&mut self,
|
||||
new_sk_id: NodeId,
|
||||
new_wal_source_connconf: PgConnectionConfig,
|
||||
ctx: &RequestContext,
|
||||
) {
|
||||
self.drop_old_connection(true).await;
|
||||
|
||||
let id = self.id;
|
||||
let connect_timeout = self.wal_connect_timeout;
|
||||
let timeline = Arc::clone(&self.timeline);
|
||||
let ctx = ctx.detached_child(
|
||||
TaskKind::WalReceiverConnectionHandler,
|
||||
ctx.download_behavior(),
|
||||
);
|
||||
let connection_handle = TaskHandle::spawn(move |events_sender, cancellation| {
|
||||
async move {
|
||||
super::walreceiver_connection::handle_walreceiver_connection(
|
||||
@@ -415,6 +425,7 @@ impl WalreceiverState {
|
||||
events_sender,
|
||||
cancellation,
|
||||
connect_timeout,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
.context("walreceiver connection handling failure")
|
||||
@@ -1245,18 +1256,18 @@ mod tests {
|
||||
const DUMMY_SAFEKEEPER_HOST: &str = "safekeeper_connstr";
|
||||
|
||||
async fn dummy_state(harness: &TenantHarness<'_>) -> WalreceiverState {
|
||||
let (tenant, ctx) = harness.load().await;
|
||||
let timeline = tenant
|
||||
.create_empty_timeline(TIMELINE_ID, Lsn(0), crate::DEFAULT_PG_VERSION, &ctx)
|
||||
.expect("Failed to create an empty timeline for dummy wal connection manager");
|
||||
let timeline = timeline.initialize(&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(),
|
||||
|
||||
@@ -22,6 +22,7 @@ use tokio_postgres::{replication::ReplicationStream, Client};
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use tracing::{debug, error, info, trace, warn};
|
||||
|
||||
use crate::context::RequestContext;
|
||||
use crate::{metrics::LIVE_CONNECTIONS_COUNT, walreceiver::TaskStateUpdate};
|
||||
use crate::{
|
||||
task_mgr,
|
||||
@@ -62,6 +63,7 @@ pub async fn handle_walreceiver_connection(
|
||||
events_sender: watch::Sender<TaskStateUpdate<WalConnectionStatus>>,
|
||||
cancellation: CancellationToken,
|
||||
connect_timeout: Duration,
|
||||
ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// Connect to the database in replication mode.
|
||||
info!("connecting to {wal_source_connconf:?}");
|
||||
@@ -103,10 +105,14 @@ pub async fn handle_walreceiver_connection(
|
||||
|
||||
// The connection object performs the actual communication with the database,
|
||||
// so spawn it off to run on its own.
|
||||
let _connection_ctx = ctx.detached_child(
|
||||
TaskKind::WalReceiverConnectionPoller,
|
||||
ctx.download_behavior(),
|
||||
);
|
||||
let connection_cancellation = cancellation.clone();
|
||||
task_mgr::spawn(
|
||||
WALRECEIVER_RUNTIME.handle(),
|
||||
TaskKind::WalReceiverConnection,
|
||||
TaskKind::WalReceiverConnectionPoller,
|
||||
Some(timeline.tenant_id),
|
||||
Some(timeline.timeline_id),
|
||||
"walreceiver connection",
|
||||
@@ -121,7 +127,7 @@ pub async fn handle_walreceiver_connection(
|
||||
}
|
||||
}
|
||||
},
|
||||
|
||||
// Future: replace connection_cancellation with connection_ctx cancellation
|
||||
_ = connection_cancellation.cancelled() => info!("Connection cancelled"),
|
||||
}
|
||||
Ok(())
|
||||
@@ -184,7 +190,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! {
|
||||
@@ -255,7 +261,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}"))?;
|
||||
|
||||
@@ -333,7 +339,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,
|
||||
|
||||
Reference in New Issue
Block a user