On-Demand Download

The code in this change was extracted from #2595 (Heikki’s on-demand
download draft PR).

High-Level Changes

- New RemoteLayer Type
- On-Demand Download As An Effect Of Page Reconstruction
- Breaking Semantics For Physical Size Metrics

There are several follow-up work items planned.
Refer to the Epic issue on GitHub: https://github.com/neondatabase/neon/issues/2029

closes https://github.com/neondatabase/neon/pull/3013

Co-authored-by: Kirill Bulatov <kirill@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>

New RemoteLayer Type
====================

Instead of downloading all layers during tenant attach, we create
RemoteLayer instances for each of them and add them to the layer map.

On-Demand Download As An Effect Of Page Reconstruction
======================================================

At the heart of pageserver is Timeline::get_reconstruct_data(). It
traverses the layer map until it has collected all the data it needs to
produce the page image. Most code in the code base uses it, though many
layers of indirection.

Before this patch, the function would use synchronous filesystem IO to
load data from disk-resident layer files if the data was not cached.

That is not possible with RemoteLayer, because the layer file has not
been downloaded yet. So, we do the download when get_reconstruct_data
gets there, i.e., “on demand”.

The mechanics of how the download is done are rather involved, because
of the infamous async-sync-async sandwich problem that plagues the async
Rust world. We use the new PageReconstructResult type to work around
this. Its introduction is the cause for a good amount of code churn in
this patch. Refer to the block comment on `with_ondemand_download()`
for details.

Breaking Semantics For Physical Size Metrics
============================================

We rename prometheus metric pageserver_{current,resident}_physical_size to
reflect what this metric actually represents with on-demand download.
This intentionally BREAKS existing grafana dashboard and the cost model data
pipeline. Breaking is desirable because the meaning of this metrics has changed
with on-demand download. See
 https://docs.google.com/document/d/12AFpvKY-7FZdR5a4CaD6Ir_rI3QokdCLSPJ6upHxJBo/edit#
for how we will handle this breakage.

Likewise, we rename the new billing_metrics’s PhysicalSize => ResidentSize.
This is not yet used anywhere, so, this is not a breaking change.

There is still a field called TimelineInfo::current_physical_size. It
is now the sum of the layer sizes in layer map, regardless of whether
local or remote. To compute that sum, we added a new trait method
PersistentLayer::file_size().

When updating the Python tests, we got rid of
current_physical_size_non_incremental. An earlier commit removed it from
the OpenAPI spec already, so this is not a breaking change.

test_timeline_size.py has grown additional assertions on the
resident_physical_size metric.
This commit is contained in:
Heikki Linnakangas
2022-12-05 10:20:24 -05:00
committed by Christian Schwarz
parent 31543c4acc
commit 7ff591ffbf
36 changed files with 2556 additions and 711 deletions

View File

@@ -22,7 +22,8 @@ use std::time::SystemTime;
use tar::{Builder, EntryType, Header};
use tracing::*;
use crate::tenant::Timeline;
use crate::task_mgr;
use crate::tenant::{with_ondemand_download, PageReconstructResult, Timeline};
use pageserver_api::reltag::{RelTag, SlruKind};
use postgres_ffi::pg_constants::{DEFAULTTABLESPACE_OID, GLOBALTABLESPACE_OID};
@@ -152,23 +153,29 @@ where
SlruKind::MultiXactOffsets,
SlruKind::MultiXactMembers,
] {
for segno in self.timeline.list_slru_segments(kind, self.lsn)? {
for segno in
with_ondemand_download_sync(|| self.timeline.list_slru_segments(kind, self.lsn))?
{
self.add_slru_segment(kind, segno)?;
}
}
// Create tablespace directories
for ((spcnode, dbnode), has_relmap_file) in self.timeline.list_dbdirs(self.lsn)? {
for ((spcnode, dbnode), has_relmap_file) in
with_ondemand_download_sync(|| self.timeline.list_dbdirs(self.lsn))?
{
self.add_dbdir(spcnode, dbnode, has_relmap_file)?;
// 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)? {
for rel in with_ondemand_download_sync(|| {
self.timeline.list_rels(spcnode, dbnode, self.lsn)
})? {
self.add_rel(rel)?;
}
}
}
for xid in self.timeline.list_twophase_files(self.lsn)? {
for xid in with_ondemand_download_sync(|| self.timeline.list_twophase_files(self.lsn))? {
self.add_twophase_file(xid)?;
}
@@ -185,7 +192,8 @@ where
}
fn add_rel(&mut self, tag: RelTag) -> anyhow::Result<()> {
let nblocks = self.timeline.get_rel_size(tag, self.lsn, false)?;
let nblocks =
with_ondemand_download_sync(|| self.timeline.get_rel_size(tag, self.lsn, false))?;
// Function that adds relation segment data to archive
let mut add_file = |segment_index, data: &Vec<u8>| -> anyhow::Result<()> {
@@ -208,7 +216,8 @@ where
for blknum in blocks {
let img = self
.timeline
.get_rel_page_at_lsn(tag, blknum, self.lsn, false)?;
.get_rel_page_at_lsn(tag, blknum, self.lsn, false)
.no_ondemand_download()?;
segment_data.extend_from_slice(&img[..]);
}
@@ -222,13 +231,16 @@ where
// Generate SLRU segment files from repository.
//
fn add_slru_segment(&mut self, slru: SlruKind, segno: u32) -> anyhow::Result<()> {
let nblocks = self.timeline.get_slru_segment_size(slru, segno, self.lsn)?;
let nblocks = with_ondemand_download_sync(|| {
self.timeline.get_slru_segment_size(slru, segno, self.lsn)
})?;
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)?;
let img = with_ondemand_download_sync(|| {
self.timeline
.get_slru_page_at_lsn(slru, segno, blknum, self.lsn)
})?;
if slru == SlruKind::Clog {
ensure!(img.len() == BLCKSZ as usize || img.len() == BLCKSZ as usize + 8);
@@ -260,7 +272,9 @@ where
has_relmap_file: bool,
) -> anyhow::Result<()> {
let relmap_img = if has_relmap_file {
let img = self.timeline.get_relmap_file(spcnode, dbnode, self.lsn)?;
let img = with_ondemand_download_sync(|| {
self.timeline.get_relmap_file(spcnode, dbnode, self.lsn)
})?;
ensure!(img.len() == 512);
Some(img)
} else {
@@ -295,7 +309,8 @@ where
if !has_relmap_file
&& self
.timeline
.list_rels(spcnode, dbnode, self.lsn)?
.list_rels(spcnode, dbnode, self.lsn)
.no_ondemand_download()?
.is_empty()
{
return Ok(());
@@ -327,7 +342,7 @@ where
// Extract twophase state files
//
fn add_twophase_file(&mut self, xid: TransactionId) -> anyhow::Result<()> {
let img = self.timeline.get_twophase_file(xid, self.lsn)?;
let img = with_ondemand_download_sync(|| self.timeline.get_twophase_file(xid, self.lsn))?;
let mut buf = BytesMut::new();
buf.extend_from_slice(&img[..]);
@@ -361,14 +376,12 @@ where
zenith_signal.as_bytes(),
)?;
let checkpoint_bytes = self
.timeline
.get_checkpoint(self.lsn)
.context("failed to get checkpoint bytes")?;
let pg_control_bytes = self
.timeline
.get_control_file(self.lsn)
.context("failed get control bytes")?;
let checkpoint_bytes =
with_ondemand_download_sync(|| self.timeline.get_checkpoint(self.lsn))
.context("failed to get checkpoint bytes")?;
let pg_control_bytes =
with_ondemand_download_sync(|| self.timeline.get_control_file(self.lsn))
.context("failed get control bytes")?;
let (pg_control_bytes, system_identifier) = postgres_ffi::generate_pg_control(
&pg_control_bytes,
@@ -490,3 +503,11 @@ where
}
}
}
fn with_ondemand_download_sync<F, T>(f: F) -> anyhow::Result<T>
where
F: Send + Fn() -> PageReconstructResult<T>,
T: Send,
{
task_mgr::COMPUTE_REQUEST_RUNTIME.block_on(with_ondemand_download(f))
}

View File

@@ -73,10 +73,10 @@ pub enum BillingMetricKind {
/// This is an absolute, per-tenant metric.
/// This is the same metric that tenant/tenant_id/size endpoint returns.
SyntheticStorageSize,
/// Size of all the files in the tenant's directory on disk on the pageserver.
/// Size of all the layer files in the tenant's directory on disk on the pageserver.
/// This is an absolute, per-tenant metric.
/// See also prometheus metric CURRENT_PHYSICAL_SIZE.
PhysicalSize,
/// See also prometheus metric RESIDENT_PHYSICAL_SIZE.
ResidentSize,
/// Size of the remote storage (S3) directory.
/// This is an absolute, per-tenant metric.
RemoteStorageSize,
@@ -89,7 +89,7 @@ impl FromStr for BillingMetricKind {
match s {
"written_size" => Ok(Self::WrittenSize),
"synthetic_storage_size" => Ok(Self::SyntheticStorageSize),
"physical_size" => Ok(Self::PhysicalSize),
"resident_size" => Ok(Self::ResidentSize),
"remote_storage_size" => Ok(Self::RemoteStorageSize),
_ => anyhow::bail!("invalid value \"{s}\" for metric type"),
}
@@ -101,7 +101,7 @@ impl fmt::Display for BillingMetricKind {
f.write_str(match self {
BillingMetricKind::WrittenSize => "written_size",
BillingMetricKind::SyntheticStorageSize => "synthetic_storage_size",
BillingMetricKind::PhysicalSize => "physical_size",
BillingMetricKind::ResidentSize => "resident_size",
BillingMetricKind::RemoteStorageSize => "remote_storage_size",
})
}
@@ -171,7 +171,7 @@ pub async fn collect_metrics_task(
let tenant = tenant_mgr::get_tenant(tenant_id, true).await?;
let mut tenant_physical_size = 0;
let mut tenant_resident_size = 0;
// iterate through list of timelines in tenant
for timeline in tenant.list_timelines().iter() {
@@ -186,27 +186,27 @@ pub async fn collect_metrics_task(
timeline_written_size,
));
let timeline_size = timeline.get_physical_size();
tenant_physical_size += timeline_size;
let timeline_resident_size = timeline.get_resident_physical_size();
tenant_resident_size += timeline_resident_size;
debug!(
"per-timeline current metrics for tenant: {}: timeline {} physical_size={} last_record_lsn {} (as bytes)",
tenant_id, timeline.timeline_id, timeline_size, timeline_written_size)
"per-timeline current metrics for tenant: {}: timeline {} resident_size={} last_record_lsn {} (as bytes)",
tenant_id, timeline.timeline_id, timeline_resident_size, timeline_written_size)
}
let tenant_remote_size = tenant.get_remote_size().await?;
debug!(
"collected current metrics for tenant: {}: state={:?} tenant_physical_size={} remote_size={}",
tenant_id, tenant_state, tenant_physical_size, tenant_remote_size
"collected current metrics for tenant: {}: state={:?} resident_size={} remote_size={}",
tenant_id, tenant_state, tenant_resident_size, tenant_remote_size
);
current_metrics.push((
BillingMetricsKey {
tenant_id,
timeline_id: None,
metric: BillingMetricKind::PhysicalSize,
metric: BillingMetricKind::ResidentSize,
},
tenant_physical_size,
tenant_resident_size,
));
current_metrics.push((

View File

@@ -12,7 +12,7 @@ use super::models::{
TimelineCreateRequest, TimelineInfo,
};
use crate::pgdatadir_mapping::LsnForTimestamp;
use crate::tenant::Timeline;
use crate::tenant::{with_ondemand_download, Timeline};
use crate::tenant_config::TenantConfOpt;
use crate::{config::PageServerConf, tenant_mgr};
use utils::{
@@ -78,25 +78,23 @@ fn check_permission(request: &Request<Body>, tenant_id: Option<TenantId>) -> Res
}
// Helper function to construct a TimelineInfo struct for a timeline
fn build_timeline_info(
async fn build_timeline_info(
timeline: &Arc<Timeline>,
include_non_incremental_logical_size: bool,
include_non_incremental_physical_size: bool,
) -> anyhow::Result<TimelineInfo> {
let mut info = build_timeline_info_common(timeline)?;
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
// we're executing this function, we will outlive the timeline on-disk state.
info.current_logical_size_non_incremental =
Some(timeline.get_current_logical_size_non_incremental(
info.last_record_lsn,
CancellationToken::new(),
)?);
}
if include_non_incremental_physical_size {
info.current_physical_size_non_incremental =
Some(timeline.get_physical_size_non_incremental()?)
info.current_logical_size_non_incremental = Some(
timeline
.get_current_logical_size_non_incremental(
info.last_record_lsn,
CancellationToken::new(),
)
.await?,
);
}
Ok(info)
}
@@ -128,7 +126,7 @@ fn build_timeline_info_common(timeline: &Arc<Timeline>) -> anyhow::Result<Timeli
None
}
};
let current_physical_size = Some(timeline.get_physical_size());
let current_physical_size = Some(timeline.layer_size_sum().approximate_is_ok());
let state = timeline.current_state();
let remote_consistent_lsn = timeline.get_remote_consistent_lsn().unwrap_or(Lsn(0));
@@ -145,7 +143,7 @@ fn build_timeline_info_common(timeline: &Arc<Timeline>) -> anyhow::Result<Timeli
current_logical_size,
current_physical_size,
current_logical_size_non_incremental: None,
current_physical_size_non_incremental: None,
timeline_dir_layer_file_size_sum: None,
wal_source_connstr,
last_received_msg_lsn,
last_received_msg_ts,
@@ -198,8 +196,6 @@ async fn timeline_list_handler(request: Request<Body>) -> Result<Response<Body>,
let tenant_id: TenantId = parse_request_param(&request, "tenant_id")?;
let include_non_incremental_logical_size =
query_param_present(&request, "include-non-incremental-logical-size");
let include_non_incremental_physical_size =
query_param_present(&request, "include-non-incremental-physical-size");
check_permission(&request, Some(tenant_id))?;
let response_data = async {
@@ -210,17 +206,16 @@ 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,
include_non_incremental_physical_size,
)
.context("Failed to convert tenant timeline {timeline_id} into the local one: {e:?}")
.map_err(ApiError::InternalServerError)?;
let timeline_info =
build_timeline_info(&timeline, include_non_incremental_logical_size)
.await
.context(
"Failed to convert tenant timeline {timeline_id} into the local one: {e:?}",
)
.map_err(ApiError::InternalServerError)?;
response_data.push(timeline_info);
}
Ok(response_data)
}
.instrument(info_span!("timeline_list", tenant = %tenant_id))
@@ -264,8 +259,6 @@ async fn timeline_detail_handler(request: Request<Body>) -> Result<Response<Body
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
let include_non_incremental_logical_size =
query_param_present(&request, "include-non-incremental-logical-size");
let include_non_incremental_physical_size =
query_param_present(&request, "include-non-incremental-physical-size");
check_permission(&request, Some(tenant_id))?;
let timeline_info = async {
@@ -277,13 +270,10 @@ 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,
include_non_incremental_physical_size,
)
.context("Failed to get local timeline info: {e:#}")
.map_err(ApiError::InternalServerError)?;
let timeline_info = build_timeline_info(&timeline, include_non_incremental_logical_size)
.await
.context("Failed to get local timeline info: {e:#}")
.map_err(ApiError::InternalServerError)?;
Ok::<_, ApiError>(timeline_info)
}
@@ -308,10 +298,11 @@ async fn get_lsn_by_timestamp_handler(request: Request<Body>) -> Result<Response
.await
.and_then(|tenant| tenant.get_timeline(timeline_id, true))
.map_err(ApiError::NotFound)?;
let result = match timeline
.find_lsn_for_timestamp(timestamp_pg)
.map_err(ApiError::InternalServerError)?
{
let result = with_ondemand_download(|| timeline.find_lsn_for_timestamp(timestamp_pg))
.await
.map_err(ApiError::InternalServerError)?;
let result = match result {
LsnForTimestamp::Present(lsn) => format!("{lsn}"),
LsnForTimestamp::Future(_lsn) => "future".into(),
LsnForTimestamp::Past(_lsn) => "past".into(),
@@ -433,7 +424,7 @@ async fn tenant_status(request: Request<Body>) -> Result<Response<Body>, ApiErro
// Calculate total physical size of all timelines
let mut current_physical_size = 0;
for timeline in tenant.list_timelines().iter() {
current_physical_size += timeline.get_physical_size();
current_physical_size += timeline.layer_size_sum().approximate_is_ok();
}
let state = tenant.current_state();
@@ -786,6 +777,45 @@ async fn timeline_checkpoint_handler(request: Request<Body>) -> Result<Response<
json_response(StatusCode::OK, ())
}
async fn timeline_download_remote_layers_handler_post(
request: Request<Body>,
) -> Result<Response<Body>, ApiError> {
let tenant_id: TenantId = parse_request_param(&request, "tenant_id")?;
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
check_permission(&request, Some(tenant_id))?;
let tenant = tenant_mgr::get_tenant(tenant_id, true)
.await
.map_err(ApiError::NotFound)?;
let timeline = tenant
.get_timeline(timeline_id, true)
.map_err(ApiError::NotFound)?;
match timeline.spawn_download_all_remote_layers().await {
Ok(st) => json_response(StatusCode::ACCEPTED, st),
Err(st) => json_response(StatusCode::CONFLICT, st),
}
}
async fn timeline_download_remote_layers_handler_get(
request: Request<Body>,
) -> Result<Response<Body>, ApiError> {
let tenant_id: TenantId = parse_request_param(&request, "tenant_id")?;
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
check_permission(&request, Some(tenant_id))?;
let tenant = tenant_mgr::get_tenant(tenant_id, true)
.await
.map_err(ApiError::NotFound)?;
let timeline = tenant
.get_timeline(timeline_id, true)
.map_err(ApiError::NotFound)?;
let info = timeline
.get_download_all_remote_layers_task_info()
.context("task never started since last pageserver process start")
.map_err(ApiError::NotFound)?;
json_response(StatusCode::OK, info)
}
async fn handler_404(_: Request<Body>) -> Result<Response<Body>, ApiError> {
json_response(
StatusCode::NOT_FOUND,
@@ -870,6 +900,14 @@ pub fn make_router(
"/v1/tenant/:tenant_id/timeline/:timeline_id/checkpoint",
testing_api!("run timeline checkpoint", timeline_checkpoint_handler),
)
.post(
"/v1/tenant/:tenant_id/timeline/:timeline_id/download_remote_layers",
timeline_download_remote_layers_handler_post,
)
.get(
"/v1/tenant/:tenant_id/timeline/:timeline_id/download_remote_layers",
timeline_download_remote_layers_handler_get,
)
.delete(
"/v1/tenant/:tenant_id/timeline/:timeline_id",
timeline_delete_handler,

View File

@@ -187,13 +187,13 @@ fn import_slru<Reader: Read>(
path: &Path,
mut reader: Reader,
len: usize,
) -> Result<()> {
trace!("importing slru file {}", path.display());
) -> anyhow::Result<()> {
info!("importing slru file {path:?}");
let mut buf: [u8; 8192] = [0u8; 8192];
let filename = &path
.file_name()
.expect("missing slru filename")
.with_context(|| format!("missing slru filename for path {path:?}"))?
.to_string_lossy();
let segno = u32::from_str_radix(filename, 16)?;
@@ -279,7 +279,9 @@ fn import_wal(walpath: &Path, tline: &Timeline, startpoint: Lsn, endpoint: Lsn)
let mut decoded = DecodedWALRecord::default();
while last_lsn <= endpoint {
if let Some((lsn, recdata)) = waldecoder.poll_decode()? {
walingest.ingest_record(recdata, lsn, &mut modification, &mut decoded)?;
walingest
.ingest_record(recdata, lsn, &mut modification, &mut decoded)
.no_ondemand_download()?;
last_lsn = lsn;
nrecords += 1;
@@ -405,7 +407,9 @@ pub fn import_wal_from_tar<Reader: Read>(
let mut decoded = DecodedWALRecord::default();
while last_lsn <= end_lsn {
if let Some((lsn, recdata)) = waldecoder.poll_decode()? {
walingest.ingest_record(recdata, lsn, &mut modification, &mut decoded)?;
walingest
.ingest_record(recdata, lsn, &mut modification, &mut decoded)
.no_ondemand_download()?;
last_lsn = lsn;
debug!("imported record at {} (end {})", lsn, end_lsn);

View File

@@ -91,7 +91,7 @@ async fn exponential_backoff(n: u32, base_increment: f64, max_seconds: f64) {
}
}
fn exponential_backoff_duration_seconds(n: u32, base_increment: f64, max_seconds: f64) -> f64 {
pub fn exponential_backoff_duration_seconds(n: u32, base_increment: f64, max_seconds: f64) -> f64 {
if n == 0 {
0.0
} else {

View File

@@ -84,13 +84,10 @@ static LAST_RECORD_LSN: Lazy<IntGaugeVec> = Lazy::new(|| {
.expect("failed to define a metric")
});
// Metrics for determining timeline's physical size.
// A layered timeline's physical is defined as the total size of
// (delta/image) layer files on disk.
static CURRENT_PHYSICAL_SIZE: Lazy<UIntGaugeVec> = Lazy::new(|| {
static RESIDENT_PHYSICAL_SIZE: Lazy<UIntGaugeVec> = Lazy::new(|| {
register_uint_gauge_vec!(
"pageserver_current_physical_size",
"Current physical size grouped by timeline",
"pageserver_resident_physical_size",
"The size of the layer files present in the pageserver's filesystem.",
&["tenant_id", "timeline_id"]
)
.expect("failed to define a metric")
@@ -146,8 +143,9 @@ const STORAGE_IO_TIME_BUCKETS: &[f64] = &[
1.0, // 1 sec
];
const STORAGE_IO_TIME_OPERATIONS: &[&str] =
&["open", "close", "read", "write", "seek", "fsync", "gc"];
const STORAGE_IO_TIME_OPERATIONS: &[&str] = &[
"open", "close", "read", "write", "seek", "fsync", "gc", "metadata",
];
const STORAGE_IO_SIZE_OPERATIONS: &[&str] = &["read", "write"];
@@ -375,7 +373,7 @@ pub struct TimelineMetrics {
pub load_layer_map_histo: Histogram,
pub last_record_gauge: IntGauge,
pub wait_lsn_time_histo: Histogram,
pub current_physical_size_gauge: UIntGauge,
pub resident_physical_size_gauge: UIntGauge,
/// copy of LayeredTimeline.current_logical_size
pub current_logical_size_gauge: UIntGauge,
pub num_persistent_files_created: IntCounter,
@@ -416,7 +414,7 @@ impl TimelineMetrics {
let wait_lsn_time_histo = WAIT_LSN_TIME
.get_metric_with_label_values(&[&tenant_id, &timeline_id])
.unwrap();
let current_physical_size_gauge = CURRENT_PHYSICAL_SIZE
let resident_physical_size_gauge = RESIDENT_PHYSICAL_SIZE
.get_metric_with_label_values(&[&tenant_id, &timeline_id])
.unwrap();
let current_logical_size_gauge = CURRENT_LOGICAL_SIZE
@@ -442,7 +440,7 @@ impl TimelineMetrics {
load_layer_map_histo,
last_record_gauge,
wait_lsn_time_histo,
current_physical_size_gauge,
resident_physical_size_gauge,
current_logical_size_gauge,
num_persistent_files_created,
persistent_bytes_written,
@@ -458,7 +456,7 @@ impl Drop for TimelineMetrics {
let _ = MATERIALIZED_PAGE_CACHE_HIT.remove_label_values(&[tenant_id, timeline_id]);
let _ = LAST_RECORD_LSN.remove_label_values(&[tenant_id, timeline_id]);
let _ = WAIT_LSN_TIME.remove_label_values(&[tenant_id, timeline_id]);
let _ = CURRENT_PHYSICAL_SIZE.remove_label_values(&[tenant_id, timeline_id]);
let _ = RESIDENT_PHYSICAL_SIZE.remove_label_values(&[tenant_id, timeline_id]);
let _ = CURRENT_LOGICAL_SIZE.remove_label_values(&[tenant_id, timeline_id]);
let _ = NUM_PERSISTENT_FILES_CREATED.remove_label_values(&[tenant_id, timeline_id]);
let _ = PERSISTENT_BYTES_WRITTEN.remove_label_values(&[tenant_id, timeline_id]);

View File

@@ -541,7 +541,10 @@ impl PageServerHandler {
let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn)
.await?;
let exists = timeline.get_rel_exists(req.rel, lsn, req.latest)?;
let exists = crate::tenant::with_ondemand_download(|| {
timeline.get_rel_exists(req.rel, lsn, req.latest)
})
.await?;
Ok(PagestreamBeMessage::Exists(PagestreamExistsResponse {
exists,
@@ -558,7 +561,10 @@ impl PageServerHandler {
let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn)
.await?;
let n_blocks = timeline.get_rel_size(req.rel, lsn, req.latest)?;
let n_blocks = crate::tenant::with_ondemand_download(|| {
timeline.get_rel_size(req.rel, lsn, req.latest)
})
.await?;
Ok(PagestreamBeMessage::Nblocks(PagestreamNblocksResponse {
n_blocks,
@@ -575,9 +581,10 @@ impl PageServerHandler {
let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn)
.await?;
let total_blocks =
timeline.get_db_size(DEFAULTTABLESPACE_OID, req.dbnode, lsn, req.latest)?;
let total_blocks = crate::tenant::with_ondemand_download(|| {
timeline.get_db_size(DEFAULTTABLESPACE_OID, req.dbnode, lsn, req.latest)
})
.await?;
let db_size = total_blocks as i64 * BLCKSZ as i64;
Ok(PagestreamBeMessage::DbSize(PagestreamDbSizeResponse {
@@ -603,11 +610,14 @@ impl PageServerHandler {
}
*/
// FIXME: this profiling now happens at different place than it used to. The
// current profiling is based on a thread-local variable, so it doesn't work
// across awaits
let _profiling_guard = profpoint_start(self.conf, ProfilingConfig::PageRequests);
let page = timeline.get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest)?;
let page = crate::tenant::with_ondemand_download(|| {
// FIXME: this profiling now happens at different place than it used to. The
// current profiling is based on a thread-local variable, so it doesn't work
// across awaits
let _profiling_guard = profpoint_start(self.conf, ProfilingConfig::PageRequests);
timeline.get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest)
})
.await?;
Ok(PagestreamBeMessage::GetPage(PagestreamGetPageResponse {
page,

View File

@@ -6,11 +6,12 @@
//! walingest.rs handles a few things like implicit relation creation and extension.
//! Clarify that)
//!
use super::tenant::PageReconstructResult;
use crate::keyspace::{KeySpace, KeySpaceAccum};
use crate::repository::*;
use crate::tenant::Timeline;
use crate::walrecord::NeonWalRecord;
use anyhow::{self, bail, ensure, Context};
use crate::{repository::*, try_no_ondemand_download};
use anyhow::Context;
use bytes::{Buf, Bytes};
use pageserver_api::reltag::{RelTag, SlruKind};
use postgres_ffi::relfile_utils::{FSM_FORKNUM, VISIBILITYMAP_FORKNUM};
@@ -97,16 +98,18 @@ impl Timeline {
blknum: BlockNumber,
lsn: Lsn,
latest: bool,
) -> anyhow::Result<Bytes> {
ensure!(tag.relnode != 0, "invalid relnode");
) -> PageReconstructResult<Bytes> {
if tag.relnode == 0 {
return PageReconstructResult::from(anyhow::anyhow!("invalid relnode"));
}
let nblocks = self.get_rel_size(tag, lsn, latest)?;
let nblocks = try_no_ondemand_download!(self.get_rel_size(tag, lsn, latest));
if blknum >= nblocks {
debug!(
"read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page",
tag, blknum, lsn, nblocks
);
return Ok(ZERO_PAGE.clone());
return PageReconstructResult::Success(ZERO_PAGE.clone());
}
let key = rel_block_to_key(tag, blknum);
@@ -120,38 +123,45 @@ impl Timeline {
dbnode: Oid,
lsn: Lsn,
latest: bool,
) -> anyhow::Result<usize> {
) -> PageReconstructResult<usize> {
let mut total_blocks = 0;
let rels = self.list_rels(spcnode, dbnode, lsn)?;
let rels = try_no_ondemand_download!(self.list_rels(spcnode, dbnode, lsn));
for rel in rels {
let n_blocks = self.get_rel_size(rel, lsn, latest)?;
let n_blocks = try_no_ondemand_download!(self.get_rel_size(rel, lsn, latest));
total_blocks += n_blocks as usize;
}
Ok(total_blocks)
PageReconstructResult::Success(total_blocks)
}
/// Get size of a relation file
pub fn get_rel_size(&self, tag: RelTag, lsn: Lsn, latest: bool) -> anyhow::Result<BlockNumber> {
ensure!(tag.relnode != 0, "invalid relnode");
pub fn get_rel_size(
&self,
tag: RelTag,
lsn: Lsn,
latest: bool,
) -> PageReconstructResult<BlockNumber> {
if tag.relnode == 0 {
return PageReconstructResult::from(anyhow::anyhow!("invalid relnode"));
}
if let Some(nblocks) = self.get_cached_rel_size(&tag, lsn) {
return Ok(nblocks);
return PageReconstructResult::Success(nblocks);
}
if (tag.forknum == FSM_FORKNUM || tag.forknum == VISIBILITYMAP_FORKNUM)
&& !self.get_rel_exists(tag, lsn, latest)?
&& !try_no_ondemand_download!(self.get_rel_exists(tag, lsn, latest))
{
// FIXME: Postgres sometimes calls smgrcreate() to create
// FSM, and smgrnblocks() on it immediately afterwards,
// without extending it. Tolerate that by claiming that
// any non-existent FSM fork has size 0.
return Ok(0);
return PageReconstructResult::Success(0);
}
let key = rel_size_to_key(tag);
let mut buf = self.get(key, lsn)?;
let mut buf = try_no_ondemand_download!(self.get(key, lsn));
let nblocks = buf.get_u32_le();
if latest {
@@ -164,25 +174,35 @@ impl Timeline {
// associated with most recent value of LSN.
self.update_cached_rel_size(tag, lsn, nblocks);
}
Ok(nblocks)
PageReconstructResult::Success(nblocks)
}
/// Does relation exist?
pub fn get_rel_exists(&self, tag: RelTag, lsn: Lsn, _latest: bool) -> anyhow::Result<bool> {
ensure!(tag.relnode != 0, "invalid relnode");
pub fn get_rel_exists(
&self,
tag: RelTag,
lsn: Lsn,
_latest: bool,
) -> PageReconstructResult<bool> {
if tag.relnode == 0 {
return PageReconstructResult::from(anyhow::anyhow!("invalid relnode"));
}
// first try to lookup relation in cache
if let Some(_nblocks) = self.get_cached_rel_size(&tag, lsn) {
return Ok(true);
return PageReconstructResult::Success(true);
}
// fetch directory listing
let key = rel_dir_to_key(tag.spcnode, tag.dbnode);
let buf = self.get(key, lsn)?;
let dir = RelDirectory::des(&buf)?;
let buf = try_no_ondemand_download!(self.get(key, lsn));
let exists = dir.rels.get(&(tag.relnode, tag.forknum)).is_some();
Ok(exists)
match RelDirectory::des(&buf).context("deserialization failure") {
Ok(dir) => {
let exists = dir.rels.get(&(tag.relnode, tag.forknum)).is_some();
PageReconstructResult::Success(exists)
}
Err(e) => PageReconstructResult::from(e),
}
}
/// Get a list of all existing relations in given tablespace and database.
@@ -191,21 +211,25 @@ impl Timeline {
spcnode: Oid,
dbnode: Oid,
lsn: Lsn,
) -> anyhow::Result<HashSet<RelTag>> {
) -> PageReconstructResult<HashSet<RelTag>> {
// fetch directory listing
let key = rel_dir_to_key(spcnode, dbnode);
let buf = self.get(key, lsn)?;
let dir = RelDirectory::des(&buf)?;
let buf = try_no_ondemand_download!(self.get(key, lsn));
let rels: HashSet<RelTag> =
HashSet::from_iter(dir.rels.iter().map(|(relnode, forknum)| RelTag {
spcnode,
dbnode,
relnode: *relnode,
forknum: *forknum,
}));
match RelDirectory::des(&buf).context("deserialization failure") {
Ok(dir) => {
let rels: HashSet<RelTag> =
HashSet::from_iter(dir.rels.iter().map(|(relnode, forknum)| RelTag {
spcnode,
dbnode,
relnode: *relnode,
forknum: *forknum,
}));
Ok(rels)
PageReconstructResult::Success(rels)
}
Err(e) => PageReconstructResult::from(e),
}
}
/// Look up given SLRU page version.
@@ -215,7 +239,7 @@ impl Timeline {
segno: u32,
blknum: BlockNumber,
lsn: Lsn,
) -> anyhow::Result<Bytes> {
) -> PageReconstructResult<Bytes> {
let key = slru_block_to_key(kind, segno, blknum);
self.get(key, lsn)
}
@@ -226,10 +250,10 @@ impl Timeline {
kind: SlruKind,
segno: u32,
lsn: Lsn,
) -> anyhow::Result<BlockNumber> {
) -> PageReconstructResult<BlockNumber> {
let key = slru_segment_size_to_key(kind, segno);
let mut buf = self.get(key, lsn)?;
Ok(buf.get_u32_le())
let mut buf = try_no_ondemand_download!(self.get(key, lsn));
PageReconstructResult::Success(buf.get_u32_le())
}
/// Get size of an SLRU segment
@@ -238,14 +262,18 @@ impl Timeline {
kind: SlruKind,
segno: u32,
lsn: Lsn,
) -> anyhow::Result<bool> {
) -> PageReconstructResult<bool> {
// fetch directory listing
let key = slru_dir_to_key(kind);
let buf = self.get(key, lsn)?;
let dir = SlruSegmentDirectory::des(&buf)?;
let buf = try_no_ondemand_download!(self.get(key, lsn));
let exists = dir.segments.get(&segno).is_some();
Ok(exists)
match SlruSegmentDirectory::des(&buf).context("deserialization failure") {
Ok(dir) => {
let exists = dir.segments.get(&segno).is_some();
PageReconstructResult::Success(exists)
}
Err(e) => PageReconstructResult::from(e),
}
}
/// Locate LSN, such that all transactions that committed before
@@ -258,7 +286,7 @@ impl Timeline {
pub fn find_lsn_for_timestamp(
&self,
search_timestamp: TimestampTz,
) -> anyhow::Result<LsnForTimestamp> {
) -> PageReconstructResult<LsnForTimestamp> {
let gc_cutoff_lsn_guard = self.get_latest_gc_cutoff_lsn();
let min_lsn = *gc_cutoff_lsn_guard;
let max_lsn = self.get_last_record_lsn();
@@ -274,12 +302,12 @@ impl Timeline {
// cannot overflow, high and low are both smaller than u64::MAX / 2
let mid = (high + low) / 2;
let cmp = self.is_latest_commit_timestamp_ge_than(
let cmp = try_no_ondemand_download!(self.is_latest_commit_timestamp_ge_than(
search_timestamp,
Lsn(mid * 8),
&mut found_smaller,
&mut found_larger,
)?;
));
if cmp {
high = mid;
@@ -291,15 +319,15 @@ impl Timeline {
(false, false) => {
// This can happen if no commit records have been processed yet, e.g.
// just after importing a cluster.
Ok(LsnForTimestamp::NoData(max_lsn))
PageReconstructResult::Success(LsnForTimestamp::NoData(max_lsn))
}
(true, false) => {
// Didn't find any commit timestamps larger than the request
Ok(LsnForTimestamp::Future(max_lsn))
PageReconstructResult::Success(LsnForTimestamp::Future(max_lsn))
}
(false, true) => {
// Didn't find any commit timestamps smaller than the request
Ok(LsnForTimestamp::Past(max_lsn))
PageReconstructResult::Success(LsnForTimestamp::Past(max_lsn))
}
(true, true) => {
// low is the LSN of the first commit record *after* the search_timestamp,
@@ -309,7 +337,7 @@ impl Timeline {
// Otherwise, if you restore to the returned LSN, the database will
// include physical changes from later commits that will be marked
// as aborted, and will need to be vacuumed away.
Ok(LsnForTimestamp::Present(Lsn((low - 1) * 8)))
PageReconstructResult::Success(LsnForTimestamp::Present(Lsn((low - 1) * 8)))
}
}
}
@@ -327,12 +355,20 @@ impl Timeline {
probe_lsn: Lsn,
found_smaller: &mut bool,
found_larger: &mut bool,
) -> anyhow::Result<bool> {
for segno in self.list_slru_segments(SlruKind::Clog, probe_lsn)? {
let nblocks = self.get_slru_segment_size(SlruKind::Clog, segno, probe_lsn)?;
) -> PageReconstructResult<bool> {
for segno in try_no_ondemand_download!(self.list_slru_segments(SlruKind::Clog, probe_lsn)) {
let nblocks = try_no_ondemand_download!(self.get_slru_segment_size(
SlruKind::Clog,
segno,
probe_lsn
));
for blknum in (0..nblocks).rev() {
let clog_page =
self.get_slru_page_at_lsn(SlruKind::Clog, segno, blknum, probe_lsn)?;
let clog_page = try_no_ondemand_download!(self.get_slru_page_at_lsn(
SlruKind::Clog,
segno,
blknum,
probe_lsn
));
if clog_page.len() == BLCKSZ as usize + 8 {
let mut timestamp_bytes = [0u8; 8];
@@ -341,61 +377,75 @@ impl Timeline {
if timestamp >= search_timestamp {
*found_larger = true;
return Ok(true);
return PageReconstructResult::Success(true);
} else {
*found_smaller = true;
}
}
}
}
Ok(false)
PageReconstructResult::Success(false)
}
/// Get a list of SLRU segments
pub fn list_slru_segments(&self, kind: SlruKind, lsn: Lsn) -> anyhow::Result<HashSet<u32>> {
pub fn list_slru_segments(
&self,
kind: SlruKind,
lsn: Lsn,
) -> PageReconstructResult<HashSet<u32>> {
// fetch directory entry
let key = slru_dir_to_key(kind);
let buf = self.get(key, lsn)?;
let dir = SlruSegmentDirectory::des(&buf)?;
Ok(dir.segments)
let buf = try_no_ondemand_download!(self.get(key, lsn));
match SlruSegmentDirectory::des(&buf).context("deserialization failure") {
Ok(dir) => PageReconstructResult::Success(dir.segments),
Err(e) => PageReconstructResult::from(e),
}
}
pub fn get_relmap_file(&self, spcnode: Oid, dbnode: Oid, lsn: Lsn) -> anyhow::Result<Bytes> {
pub fn get_relmap_file(
&self,
spcnode: Oid,
dbnode: Oid,
lsn: Lsn,
) -> PageReconstructResult<Bytes> {
let key = relmap_file_key(spcnode, dbnode);
let buf = self.get(key, lsn)?;
Ok(buf)
let buf = try_no_ondemand_download!(self.get(key, lsn));
PageReconstructResult::Success(buf)
}
pub fn list_dbdirs(&self, lsn: Lsn) -> anyhow::Result<HashMap<(Oid, Oid), bool>> {
pub fn list_dbdirs(&self, lsn: Lsn) -> PageReconstructResult<HashMap<(Oid, Oid), bool>> {
// fetch directory entry
let buf = self.get(DBDIR_KEY, lsn)?;
let dir = DbDirectory::des(&buf)?;
let buf = try_no_ondemand_download!(self.get(DBDIR_KEY, lsn));
Ok(dir.dbdirs)
match DbDirectory::des(&buf).context("deserialization failure") {
Ok(dir) => PageReconstructResult::Success(dir.dbdirs),
Err(e) => PageReconstructResult::from(e),
}
}
pub fn get_twophase_file(&self, xid: TransactionId, lsn: Lsn) -> anyhow::Result<Bytes> {
pub fn get_twophase_file(&self, xid: TransactionId, lsn: Lsn) -> PageReconstructResult<Bytes> {
let key = twophase_file_key(xid);
let buf = self.get(key, lsn)?;
Ok(buf)
let buf = try_no_ondemand_download!(self.get(key, lsn));
PageReconstructResult::Success(buf)
}
pub fn list_twophase_files(&self, lsn: Lsn) -> anyhow::Result<HashSet<TransactionId>> {
pub fn list_twophase_files(&self, lsn: Lsn) -> PageReconstructResult<HashSet<TransactionId>> {
// fetch directory entry
let buf = self.get(TWOPHASEDIR_KEY, lsn)?;
let dir = TwoPhaseDirectory::des(&buf)?;
let buf = try_no_ondemand_download!(self.get(TWOPHASEDIR_KEY, lsn));
Ok(dir.xids)
match TwoPhaseDirectory::des(&buf).context("deserialization failure") {
Ok(dir) => PageReconstructResult::Success(dir.xids),
Err(e) => PageReconstructResult::from(e),
}
}
pub fn get_control_file(&self, lsn: Lsn) -> anyhow::Result<Bytes> {
pub fn get_control_file(&self, lsn: Lsn) -> PageReconstructResult<Bytes> {
self.get(CONTROLFILE_KEY, lsn)
}
pub fn get_checkpoint(&self, lsn: Lsn) -> anyhow::Result<Bytes> {
pub fn get_checkpoint(&self, lsn: Lsn) -> PageReconstructResult<Bytes> {
self.get(CHECKPOINT_KEY, lsn)
}
@@ -404,23 +454,26 @@ impl Timeline {
///
/// Only relation blocks are counted currently. That excludes metadata,
/// SLRUs, twophase files etc.
pub fn get_current_logical_size_non_incremental(
pub async fn get_current_logical_size_non_incremental(
&self,
lsn: Lsn,
cancel: CancellationToken,
) -> std::result::Result<u64, CalculateLogicalSizeError> {
) -> Result<u64, CalculateLogicalSizeError> {
// Fetch list of database dirs and iterate them
let buf = self.get(DBDIR_KEY, lsn)?;
let buf = self.get_download(DBDIR_KEY, lsn).await?;
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)? {
for rel in
crate::tenant::with_ondemand_download(|| self.list_rels(*spcnode, *dbnode, lsn))
.await?
{
if cancel.is_cancelled() {
return Err(CalculateLogicalSizeError::Cancelled);
}
let relsize_key = rel_size_to_key(rel);
let mut buf = self.get(relsize_key, lsn)?;
let mut buf = self.get_download(relsize_key, lsn).await?;
let relsize = buf.get_u32_le();
total_size += relsize as u64;
@@ -433,7 +486,7 @@ 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 fn collect_keyspace(&self, lsn: Lsn) -> anyhow::Result<KeySpace> {
pub async fn collect_keyspace(&self, lsn: Lsn) -> anyhow::Result<KeySpace> {
// Iterate through key ranges, greedily packing them into partitions
let mut result = KeySpaceAccum::new();
@@ -441,8 +494,8 @@ impl Timeline {
result.add_key(DBDIR_KEY);
// Fetch list of database dirs and iterate them
let buf = self.get(DBDIR_KEY, lsn)?;
let dbdir = DbDirectory::des(&buf)?;
let buf = self.get_download(DBDIR_KEY, lsn).await?;
let dbdir = DbDirectory::des(&buf).context("deserialization failure")?;
let mut dbs: Vec<(Oid, Oid)> = dbdir.dbdirs.keys().cloned().collect();
dbs.sort_unstable();
@@ -451,14 +504,15 @@ 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)
.no_ondemand_download()?
.iter()
.cloned()
.collect();
rels.sort_unstable();
for rel in rels {
let relsize_key = rel_size_to_key(rel);
let mut buf = self.get(relsize_key, lsn)?;
let mut buf = self.get_download(relsize_key, lsn).await?;
let relsize = buf.get_u32_le();
result.add_range(rel_block_to_key(rel, 0)..rel_block_to_key(rel, relsize));
@@ -474,13 +528,13 @@ impl Timeline {
] {
let slrudir_key = slru_dir_to_key(kind);
result.add_key(slrudir_key);
let buf = self.get(slrudir_key, lsn)?;
let dir = SlruSegmentDirectory::des(&buf)?;
let buf = self.get_download(slrudir_key, lsn).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)?;
let mut buf = self.get_download(segsize_key, lsn).await?;
let segsize = buf.get_u32_le();
result.add_range(
@@ -492,8 +546,8 @@ impl Timeline {
// Then pg_twophase
result.add_key(TWOPHASEDIR_KEY);
let buf = self.get(TWOPHASEDIR_KEY, lsn)?;
let twophase_dir = TwoPhaseDirectory::des(&buf)?;
let buf = self.get_download(TWOPHASEDIR_KEY, lsn).await?;
let twophase_dir = TwoPhaseDirectory::des(&buf).context("deserialization failure")?;
let mut xids: Vec<TransactionId> = twophase_dir.xids.iter().cloned().collect();
xids.sort_unstable();
for xid in xids {
@@ -606,7 +660,7 @@ impl<'a> DatadirModification<'a> {
blknum: BlockNumber,
rec: NeonWalRecord,
) -> anyhow::Result<()> {
ensure!(rel.relnode != 0, "invalid relnode");
anyhow::ensure!(rel.relnode != 0, "invalid relnode");
self.put(rel_block_to_key(rel, blknum), Value::WalRecord(rec));
Ok(())
}
@@ -633,7 +687,7 @@ impl<'a> DatadirModification<'a> {
blknum: BlockNumber,
img: Bytes,
) -> anyhow::Result<()> {
ensure!(rel.relnode != 0, "invalid relnode");
anyhow::ensure!(rel.relnode != 0, "invalid relnode");
self.put(rel_block_to_key(rel, blknum), Value::Image(img));
Ok(())
}
@@ -652,7 +706,7 @@ impl<'a> DatadirModification<'a> {
/// Store a relmapper file (pg_filenode.map) in the repository
pub fn put_relmap_file(&mut self, spcnode: Oid, dbnode: Oid, img: Bytes) -> anyhow::Result<()> {
// Add it to the directory (if it doesn't exist already)
let buf = self.get(DBDIR_KEY)?;
let buf = self.get(DBDIR_KEY).no_ondemand_download()?;
let mut dbdir = DbDirectory::des(&buf)?;
let r = dbdir.dbdirs.insert((spcnode, dbnode), true);
@@ -680,10 +734,10 @@ impl<'a> DatadirModification<'a> {
pub fn put_twophase_file(&mut self, xid: TransactionId, img: Bytes) -> anyhow::Result<()> {
// Add it to the directory entry
let buf = self.get(TWOPHASEDIR_KEY)?;
let buf = self.get(TWOPHASEDIR_KEY).no_ondemand_download()?;
let mut dir = TwoPhaseDirectory::des(&buf)?;
if !dir.xids.insert(xid) {
bail!("twophase file for xid {} already exists", xid);
anyhow::bail!("twophase file for xid {} already exists", xid);
}
self.put(
TWOPHASEDIR_KEY,
@@ -707,10 +761,13 @@ impl<'a> DatadirModification<'a> {
pub fn drop_dbdir(&mut self, spcnode: Oid, dbnode: Oid) -> anyhow::Result<()> {
let req_lsn = self.tline.get_last_record_lsn();
let total_blocks = self.tline.get_db_size(spcnode, dbnode, req_lsn, true)?;
let total_blocks = self
.tline
.get_db_size(spcnode, dbnode, req_lsn, true)
.no_ondemand_download()?;
// Remove entry from dbdir
let buf = self.get(DBDIR_KEY)?;
let buf = self.get(DBDIR_KEY).no_ondemand_download()?;
let mut dir = DbDirectory::des(&buf)?;
if dir.dbdirs.remove(&(spcnode, dbnode)).is_some() {
let buf = DbDirectory::ser(&dir)?;
@@ -734,10 +791,10 @@ impl<'a> DatadirModification<'a> {
///
/// 'nblocks' is the initial size.
pub fn put_rel_creation(&mut self, rel: RelTag, nblocks: BlockNumber) -> anyhow::Result<()> {
ensure!(rel.relnode != 0, "invalid relnode");
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)?)?;
let mut dbdir = DbDirectory::des(&self.get(DBDIR_KEY).no_ondemand_download()?)?;
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
@@ -749,12 +806,12 @@ impl<'a> DatadirModification<'a> {
RelDirectory::default()
} else {
// reldir already exists, fetch it
RelDirectory::des(&self.get(rel_dir_key)?)?
RelDirectory::des(&self.get(rel_dir_key).no_ondemand_download()?)?
};
// Add the new relation to the rel directory entry, and write it back
if !rel_dir.rels.insert((rel.relnode, rel.forknum)) {
bail!("rel {} already exists", rel);
anyhow::bail!("rel {rel} already exists");
}
self.put(
rel_dir_key,
@@ -778,12 +835,16 @@ impl<'a> DatadirModification<'a> {
/// Truncate relation
pub fn put_rel_truncation(&mut self, rel: RelTag, nblocks: BlockNumber) -> anyhow::Result<()> {
ensure!(rel.relnode != 0, "invalid relnode");
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)? {
if self
.tline
.get_rel_exists(rel, last_lsn, true)
.no_ondemand_download()?
{
let size_key = rel_size_to_key(rel);
// Fetch the old size first
let old_size = self.get(size_key)?.get_u32_le();
let old_size = self.get(size_key).no_ondemand_download()?.get_u32_le();
// Update the entry with the new size.
let buf = nblocks.to_le_bytes();
@@ -804,11 +865,11 @@ impl<'a> DatadirModification<'a> {
/// Extend relation
/// If new size is smaller, do nothing.
pub fn put_rel_extend(&mut self, rel: RelTag, nblocks: BlockNumber) -> anyhow::Result<()> {
ensure!(rel.relnode != 0, "invalid relnode");
anyhow::ensure!(rel.relnode != 0, "invalid relnode");
// Put size
let size_key = rel_size_to_key(rel);
let old_size = self.get(size_key)?.get_u32_le();
let old_size = self.get(size_key).no_ondemand_download()?.get_u32_le();
// only extend relation here. never decrease the size
if nblocks > old_size {
@@ -825,11 +886,11 @@ impl<'a> DatadirModification<'a> {
/// Drop a relation.
pub fn put_rel_drop(&mut self, rel: RelTag) -> anyhow::Result<()> {
ensure!(rel.relnode != 0, "invalid relnode");
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)?;
let buf = self.get(dir_key).no_ondemand_download()?;
let mut dir = RelDirectory::des(&buf)?;
if dir.rels.remove(&(rel.relnode, rel.forknum)) {
@@ -840,7 +901,7 @@ impl<'a> DatadirModification<'a> {
// update logical size
let size_key = rel_size_to_key(rel);
let old_size = self.get(size_key)?.get_u32_le();
let old_size = self.get(size_key).no_ondemand_download()?.get_u32_le();
self.pending_nblocks -= old_size as i64;
// Remove enty from relation size cache
@@ -860,11 +921,11 @@ impl<'a> DatadirModification<'a> {
) -> anyhow::Result<()> {
// Add it to the directory entry
let dir_key = slru_dir_to_key(kind);
let buf = self.get(dir_key)?;
let buf = self.get(dir_key).no_ondemand_download()?;
let mut dir = SlruSegmentDirectory::des(&buf)?;
if !dir.segments.insert(segno) {
bail!("slru segment {:?}/{} already exists", kind, segno);
anyhow::bail!("slru segment {kind:?}/{segno} already exists");
}
self.put(
dir_key,
@@ -899,7 +960,7 @@ impl<'a> DatadirModification<'a> {
pub fn drop_slru_segment(&mut self, kind: SlruKind, segno: u32) -> anyhow::Result<()> {
// Remove it from the directory entry
let dir_key = slru_dir_to_key(kind);
let buf = self.get(dir_key)?;
let buf = self.get(dir_key).no_ondemand_download()?;
let mut dir = SlruSegmentDirectory::des(&buf)?;
if !dir.segments.remove(&segno) {
@@ -925,7 +986,7 @@ impl<'a> DatadirModification<'a> {
/// This method is used for marking truncated SLRU files
pub fn drop_twophase_file(&mut self, xid: TransactionId) -> anyhow::Result<()> {
// Remove it from the directory entry
let buf = self.get(TWOPHASEDIR_KEY)?;
let buf = self.get(TWOPHASEDIR_KEY).no_ondemand_download()?;
let mut dir = TwoPhaseDirectory::des(&buf)?;
if !dir.xids.remove(&xid) {
@@ -1019,7 +1080,7 @@ impl<'a> DatadirModification<'a> {
// Internal helper functions to batch the modifications
fn get(&self, key: Key) -> anyhow::Result<Bytes> {
fn get(&self, key: Key) -> PageReconstructResult<Bytes> {
// Have we already updated the same key? Read the pending updated
// version in that case.
//
@@ -1027,14 +1088,16 @@ impl<'a> DatadirModification<'a> {
// value that has been removed, deletion only avoids leaking storage.
if let Some(value) = self.pending_updates.get(&key) {
if let Value::Image(img) = value {
Ok(img.clone())
PageReconstructResult::Success(img.clone())
} else {
// Currently, we never need to read back a WAL record that we
// inserted in the same "transaction". All the metadata updates
// work directly with Images, and we never need to read actual
// data pages. We could handle this if we had to, by calling
// the walredo manager, but let's keep it simple for now.
bail!("unexpected pending WAL record");
return PageReconstructResult::from(anyhow::anyhow!(
"unexpected pending WAL record"
));
}
} else {
let lsn = Lsn::max(self.tline.get_last_record_lsn(), self.lsn);
@@ -1400,7 +1463,7 @@ pub fn key_to_rel_block(key: Key) -> anyhow::Result<(RelTag, BlockNumber)> {
},
key.field6,
),
_ => bail!("unexpected value kind 0x{:02x}", key.field1),
_ => anyhow::bail!("unexpected value kind 0x{:02x}", key.field1),
})
}
@@ -1426,14 +1489,14 @@ pub fn key_to_slru_block(key: Key) -> anyhow::Result<(SlruKind, u32, BlockNumber
0x00 => SlruKind::Clog,
0x01 => SlruKind::MultiXactMembers,
0x02 => SlruKind::MultiXactOffsets,
_ => bail!("unrecognized slru kind 0x{:02x}", key.field2),
_ => anyhow::bail!("unrecognized slru kind 0x{:02x}", key.field2),
};
let segno = key.field4;
let blknum = key.field6;
(kind, segno, blknum)
}
_ => bail!("unexpected value kind 0x{:02x}", key.field1),
_ => anyhow::bail!("unexpected value kind 0x{:02x}", key.field1),
})
}

View File

@@ -148,31 +148,43 @@
//! following two cases:
//! - (1) We had the file locally, deleted it locally, scheduled a remote delete,
//! but crashed before it finished remotely.
//! - (2) We never had the file locally because we were still in tenant attach
//! when we crashed. (Similar case for on-demand download in the future.)
//! - (2) We never had the file locally because we haven't on-demand downloaded
//! it yet.
//!
//! # Downloads (= Tenant Attach)
//! # Downloads
//!
//! In addition to the upload queue, [`RemoteTimelineClient`] has functions for
//! downloading files from the remote storage. Downloads are performed immediately,
//! independently of the uploads.
//! downloading files from the remote storage. Downloads are performed immediately
//! against the `RemoteStorage`, independently of the upload queue.
//!
//! When we attach a tenant, we perform the following steps:
//! - create `Tenant` object in `TenantState::Attaching` state
//! - List timelines that are present in remote storage, and download their remote [`IndexPart`]s
//! - For each timeline, create `Timeline` struct and a `RemoteTimelineClient`, and initialize the client's upload queue with its `IndexPart`
//! - eagerly download all the remote layers using the client's download APIs
//! - transition tenant from `TenantState::Attaching` to `TenantState::Active` state.
//! - List timelines that are present in remote storage, and for each:
//! - download their remote [`IndexPart`]s
//! - create `Timeline` struct and a `RemoteTimelineClient`
//! - initialize the client's upload queue with its `IndexPart`
//! - create [`RemoteLayer`] instances for layers that are referenced by `IndexPart`
//! but not present locally
//! - schedule uploads for layers that are only present locally.
//! - if the remote `IndexPart`'s metadata was newer than the metadata in
//! the local filesystem, write the remote metadata to the local filesystem
//! - After the above is done for each timeline, open the tenant for business by
//! transitioning it from `TenantState::Attaching` to `TenantState::Active` state.
//! This starts the timelines' WAL-receivers and the tenant's GC & Compaction loops.
//!
//! Most of the above happens in [`Timeline::reconcile_with_remote`].
//! Most of the above steps happen in [`Timeline::reconcile_with_remote`] or its callers.
//! We keep track of the fact that a client is in `Attaching` state in a marker
//! file on the local disk.
//! However, the distinction is moot for storage sync since we call
//! `reconcile_with_remote` for tenants both with and without the marker file.
//!
//! In the future, downloading will be done on-demand and `reconcile_with_remote`
//! will only be responsible for re-scheduling upload ops after a crash of an
//! `Active` tenant.
//! file on the local disk. This is critical because, when we restart the pageserver,
//! we do not want to do the `List timelines` step for each tenant that has already
//! been successfully attached (for performance & cost reasons).
//! Instead, for a tenant without the attach marker file, we assume that the
//! local state is in sync or ahead of the remote state. This includes the list
//! of all of the tenant's timelines, which is particularly critical to be up-to-date:
//! if there's a timeline on the remote that the pageserver doesn't know about,
//! the GC will not consider its branch point, leading to data loss.
//! So, for a tenant with the attach marker file, we know that we do not yet have
//! persisted all the remote timeline's metadata files locally. To exclude the
//! risk above, we re-run the procedure for such tenants
//!
//! # Operating Without Remote Storage
//!

View File

@@ -180,6 +180,10 @@ pub async fn list_remote_timelines<'a>(
let tenant_path = conf.timelines_path(&tenant_id);
let tenant_storage_path = conf.remote_path(&tenant_path)?;
fail::fail_point!("storage-sync-list-remote-timelines", |_| {
anyhow::bail!("storage-sync-list-remote-timelines");
});
let timelines = download_retry(
|| storage.list_prefixes(Some(&tenant_storage_path)),
&format!("list prefixes for {tenant_path:?}"),

View File

@@ -35,6 +35,7 @@
#![allow(clippy::declare_interior_mutable_const)]
use std::collections::HashMap;
use std::fmt;
use std::future::Future;
use std::panic::AssertUnwindSafe;
use std::sync::atomic::{AtomicU64, Ordering};
@@ -134,8 +135,15 @@ pub static BACKGROUND_RUNTIME: Lazy<Runtime> = Lazy::new(|| {
.expect("Failed to create background op runtime")
});
#[derive(Debug, Clone, Copy)]
pub struct PageserverTaskId(u64);
impl fmt::Display for PageserverTaskId {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
self.0.fmt(f)
}
}
/// Each task that we track is associated with a "task ID". It's just an
/// increasing number that we assign. Note that it is different from tokio::task::Id.
static NEXT_TASK_ID: AtomicU64 = AtomicU64::new(1);
@@ -198,6 +206,9 @@ pub enum TaskKind {
// Task that uploads a file to remote storage
RemoteUploadTask,
// Task that downloads a file from remote storage
RemoteDownloadTask,
// task that handles the initial downloading of all tenants
InitialLoad,
@@ -206,6 +217,9 @@ pub enum TaskKind {
// task that handhes metrics collection
MetricsCollection,
// task that drives downloading layers
DownloadAllRemoteLayers,
}
#[derive(Default)]
@@ -437,6 +451,10 @@ pub fn current_task_kind() -> Option<TaskKind> {
CURRENT_TASK.try_with(|ct| ct.kind).ok()
}
pub fn current_task_id() -> Option<PageserverTaskId> {
CURRENT_TASK.try_with(|ct| ct.task_id).ok()
}
/// A Future that can be used to check if the current task has been requested to
/// shut down.
pub async fn shutdown_watcher() {

View File

@@ -81,6 +81,7 @@ pub mod filename;
mod image_layer;
mod inmemory_layer;
pub mod layer_map;
mod remote_layer;
pub mod metadata;
mod par_fsync;
@@ -90,7 +91,7 @@ mod timeline;
pub mod size;
pub use timeline::Timeline;
pub use timeline::{with_ondemand_download, PageReconstructError, PageReconstructResult, Timeline};
// re-export this function so that page_cache.rs can use it.
pub use crate::tenant::ephemeral_file::writeback as writeback_ephemeral_file;
@@ -2780,9 +2781,18 @@ mod tests {
writer.finish_write(Lsn(0x20));
drop(writer);
assert_eq!(tline.get(*TEST_KEY, Lsn(0x10))?, TEST_IMG("foo at 0x10"));
assert_eq!(tline.get(*TEST_KEY, Lsn(0x1f))?, TEST_IMG("foo at 0x10"));
assert_eq!(tline.get(*TEST_KEY, Lsn(0x20))?, TEST_IMG("foo at 0x20"));
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x10)).no_ondemand_download()?,
TEST_IMG("foo at 0x10")
);
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x1f)).no_ondemand_download()?,
TEST_IMG("foo at 0x10")
);
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x20)).no_ondemand_download()?,
TEST_IMG("foo at 0x20")
);
Ok(())
}
@@ -2859,15 +2869,15 @@ mod tests {
// Check page contents on both branches
assert_eq!(
from_utf8(&tline.get(TEST_KEY_A, Lsn(0x40))?)?,
from_utf8(&tline.get(TEST_KEY_A, Lsn(0x40)).no_ondemand_download()?)?,
"foo at 0x40"
);
assert_eq!(
from_utf8(&newtline.get(TEST_KEY_A, Lsn(0x40))?)?,
from_utf8(&newtline.get(TEST_KEY_A, Lsn(0x40)).no_ondemand_download()?)?,
"bar at 0x40"
);
assert_eq!(
from_utf8(&newtline.get(TEST_KEY_B, Lsn(0x40))?)?,
from_utf8(&newtline.get(TEST_KEY_B, Lsn(0x40)).no_ondemand_download()?)?,
"foobar at 0x20"
);
@@ -3026,7 +3036,10 @@ mod tests {
tenant
.gc_iteration(Some(TIMELINE_ID), 0x10, Duration::ZERO)
.await?;
assert!(newtline.get(*TEST_KEY, Lsn(0x25)).is_ok());
assert!(newtline
.get(*TEST_KEY, Lsn(0x25))
.no_ondemand_download()
.is_ok());
Ok(())
}
@@ -3056,7 +3069,7 @@ mod tests {
// Check that the data is still accessible on the branch.
assert_eq!(
newtline.get(*TEST_KEY, Lsn(0x50))?,
newtline.get(*TEST_KEY, Lsn(0x50)).no_ondemand_download()?,
TEST_IMG(&format!("foo at {}", Lsn(0x40)))
);
@@ -3203,11 +3216,26 @@ mod tests {
tline.freeze_and_flush().await?;
tline.compact().await?;
assert_eq!(tline.get(*TEST_KEY, Lsn(0x10))?, TEST_IMG("foo at 0x10"));
assert_eq!(tline.get(*TEST_KEY, Lsn(0x1f))?, TEST_IMG("foo at 0x10"));
assert_eq!(tline.get(*TEST_KEY, Lsn(0x20))?, TEST_IMG("foo at 0x20"));
assert_eq!(tline.get(*TEST_KEY, Lsn(0x30))?, TEST_IMG("foo at 0x30"));
assert_eq!(tline.get(*TEST_KEY, Lsn(0x40))?, TEST_IMG("foo at 0x40"));
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x10)).no_ondemand_download()?,
TEST_IMG("foo at 0x10")
);
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x1f)).no_ondemand_download()?,
TEST_IMG("foo at 0x10")
);
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x20)).no_ondemand_download()?,
TEST_IMG("foo at 0x20")
);
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x30)).no_ondemand_download()?,
TEST_IMG("foo at 0x30")
);
assert_eq!(
tline.get(*TEST_KEY, Lsn(0x40)).no_ondemand_download()?,
TEST_IMG("foo at 0x40")
);
Ok(())
}
@@ -3315,7 +3343,7 @@ mod tests {
for (blknum, last_lsn) in updated.iter().enumerate() {
test_key.field6 = blknum as u32;
assert_eq!(
tline.get(test_key, lsn)?,
tline.get(test_key, lsn).no_ondemand_download()?,
TEST_IMG(&format!("{} at {}", blknum, last_lsn))
);
}
@@ -3401,7 +3429,7 @@ mod tests {
for (blknum, last_lsn) in updated.iter().enumerate() {
test_key.field6 = blknum as u32;
assert_eq!(
tline.get(test_key, lsn)?,
tline.get(test_key, lsn).no_ondemand_download()?,
TEST_IMG(&format!("{} at {}", blknum, last_lsn))
);
}
@@ -3476,7 +3504,7 @@ mod tests {
println!("checking [{idx}][{blknum}] at {lsn}");
test_key.field6 = blknum as u32;
assert_eq!(
tline.get(test_key, *lsn)?,
tline.get(test_key, *lsn).no_ondemand_download()?,
TEST_IMG(&format!("{idx} {blknum} at {lsn}"))
);
}

View File

@@ -39,7 +39,7 @@ use crate::{DELTA_FILE_MAGIC, STORAGE_FORMAT_VERSION};
use anyhow::{bail, ensure, Context, Result};
use rand::{distributions::Alphanumeric, Rng};
use serde::{Deserialize, Serialize};
use std::fs;
use std::fs::{self, File};
use std::io::{BufWriter, Write};
use std::io::{Seek, SeekFrom};
use std::ops::Range;
@@ -183,6 +183,8 @@ pub struct DeltaLayer {
pub key_range: Range<Key>,
pub lsn_range: Range<Lsn>,
pub file_size: u64,
inner: RwLock<DeltaLayerInner>,
}
@@ -411,6 +413,10 @@ impl PersistentLayer for DeltaLayer {
fs::remove_file(self.path())?;
Ok(())
}
fn file_size(&self) -> Option<u64> {
Some(self.file_size)
}
}
impl DeltaLayer {
@@ -535,6 +541,7 @@ impl DeltaLayer {
timeline_id: TimelineId,
tenant_id: TenantId,
filename: &DeltaFileName,
file_size: u64,
) -> DeltaLayer {
DeltaLayer {
path_or_conf: PathOrConf::Conf(conf),
@@ -542,6 +549,7 @@ impl DeltaLayer {
tenant_id,
key_range: filename.key_range.clone(),
lsn_range: filename.lsn_range.clone(),
file_size,
inner: RwLock::new(DeltaLayerInner {
loaded: false,
file: None,
@@ -554,21 +562,23 @@ impl DeltaLayer {
/// Create a DeltaLayer struct representing an existing file on disk.
///
/// This variant is only used for debugging purposes, by the 'pageserver_binutils' binary.
pub fn new_for_path<F>(path: &Path, file: F) -> Result<Self>
where
F: FileExt,
{
pub fn new_for_path(path: &Path, file: File) -> Result<Self> {
let mut summary_buf = Vec::new();
summary_buf.resize(PAGE_SZ, 0);
file.read_exact_at(&mut summary_buf, 0)?;
let summary = Summary::des_prefix(&summary_buf)?;
let metadata = file
.metadata()
.context("get file metadata to determine size")?;
Ok(DeltaLayer {
path_or_conf: PathOrConf::Path(path.to_path_buf()),
timeline_id: summary.timeline_id,
tenant_id: summary.tenant_id,
key_range: summary.key_range,
lsn_range: summary.lsn_range,
file_size: metadata.len(),
inner: RwLock::new(DeltaLayerInner {
loaded: false,
file: None,
@@ -725,6 +735,10 @@ impl DeltaLayerWriterInner {
file.seek(SeekFrom::Start(0))?;
Summary::ser_into(&summary, &mut file)?;
let metadata = file
.metadata()
.context("get file metadata to determine size")?;
// Note: Because we opened the file in write-only mode, we cannot
// reuse the same VirtualFile for reading later. That's why we don't
// set inner.file here. The first read will have to re-open it.
@@ -734,6 +748,7 @@ impl DeltaLayerWriterInner {
timeline_id: self.timeline_id,
key_range: self.key_start..key_end,
lsn_range: self.lsn_range.clone(),
file_size: metadata.len(),
inner: RwLock::new(DeltaLayerInner {
loaded: false,
file: None,

View File

@@ -36,10 +36,11 @@ use bytes::Bytes;
use hex;
use rand::{distributions::Alphanumeric, Rng};
use serde::{Deserialize, Serialize};
use std::fs;
use std::fs::{self, File};
use std::io::Write;
use std::io::{Seek, SeekFrom};
use std::ops::Range;
use std::os::unix::prelude::FileExt;
use std::path::{Path, PathBuf};
use std::sync::{RwLock, RwLockReadGuard};
use tracing::*;
@@ -105,6 +106,7 @@ pub struct ImageLayer {
pub tenant_id: TenantId,
pub timeline_id: TimelineId,
pub key_range: Range<Key>,
pub file_size: u64,
// This entry contains an image of all pages as of this LSN
pub lsn: Lsn,
@@ -228,6 +230,10 @@ impl PersistentLayer for ImageLayer {
fs::remove_file(self.path())?;
Ok(())
}
fn file_size(&self) -> Option<u64> {
Some(self.file_size)
}
}
impl ImageLayer {
@@ -344,6 +350,7 @@ impl ImageLayer {
timeline_id: TimelineId,
tenant_id: TenantId,
filename: &ImageFileName,
file_size: u64,
) -> ImageLayer {
ImageLayer {
path_or_conf: PathOrConf::Conf(conf),
@@ -351,6 +358,7 @@ impl ImageLayer {
tenant_id,
key_range: filename.key_range.clone(),
lsn: filename.lsn,
file_size,
inner: RwLock::new(ImageLayerInner {
loaded: false,
file: None,
@@ -363,21 +371,21 @@ impl ImageLayer {
/// Create an ImageLayer struct representing an existing file on disk.
///
/// This variant is only used for debugging purposes, by the 'pageserver_binutils' binary.
pub fn new_for_path<F>(path: &Path, file: F) -> Result<ImageLayer>
where
F: std::os::unix::prelude::FileExt,
{
pub fn new_for_path(path: &Path, file: File) -> Result<ImageLayer> {
let mut summary_buf = Vec::new();
summary_buf.resize(PAGE_SZ, 0);
file.read_exact_at(&mut summary_buf, 0)?;
let summary = Summary::des_prefix(&summary_buf)?;
let metadata = file
.metadata()
.context("get file metadata to determine size")?;
Ok(ImageLayer {
path_or_conf: PathOrConf::Path(path.to_path_buf()),
timeline_id: summary.timeline_id,
tenant_id: summary.tenant_id,
key_range: summary.key_range,
lsn: summary.lsn,
file_size: metadata.len(),
inner: RwLock::new(ImageLayerInner {
file: None,
loaded: false,
@@ -523,6 +531,10 @@ impl ImageLayerWriterInner {
file.seek(SeekFrom::Start(0))?;
Summary::ser_into(&summary, &mut file)?;
let metadata = file
.metadata()
.context("get metadata to determine file size")?;
// Note: Because we open the file in write-only mode, we cannot
// reuse the same VirtualFile for reading later. That's why we don't
// set inner.file here. The first read will have to re-open it.
@@ -532,6 +544,7 @@ impl ImageLayerWriterInner {
tenant_id: self.tenant_id,
key_range: self.key_range.clone(),
lsn: self.lsn,
file_size: metadata.len(),
inner: RwLock::new(ImageLayerInner {
loaded: false,
file: None,

View File

@@ -0,0 +1,212 @@
//! A RemoteLayer is an in-memory placeholder for a layer file that exists
//! in remote storage.
//!
use crate::config::PageServerConf;
use crate::repository::Key;
use crate::storage_sync::index::LayerFileMetadata;
use crate::tenant::delta_layer::DeltaLayer;
use crate::tenant::filename::{DeltaFileName, ImageFileName};
use crate::tenant::image_layer::ImageLayer;
use crate::tenant::storage_layer::{Layer, ValueReconstructResult, ValueReconstructState};
use anyhow::{bail, Result};
use std::ops::Range;
use std::path::PathBuf;
use std::sync::Arc;
use utils::{
id::{TenantId, TimelineId},
lsn::Lsn,
};
use super::filename::LayerFileName;
use super::storage_layer::{LayerIter, LayerKeyIter, PersistentLayer};
#[derive(Debug)]
pub struct RemoteLayer {
tenantid: TenantId,
timelineid: TimelineId,
key_range: Range<Key>,
lsn_range: Range<Lsn>,
pub file_name: LayerFileName,
pub layer_metadata: LayerFileMetadata,
is_delta: bool,
is_incremental: bool,
pub(crate) ongoing_download: Arc<tokio::sync::Semaphore>,
}
impl Layer for RemoteLayer {
fn get_key_range(&self) -> Range<Key> {
self.key_range.clone()
}
fn get_lsn_range(&self) -> Range<Lsn> {
self.lsn_range.clone()
}
fn get_value_reconstruct_data(
&self,
_key: Key,
_lsn_range: Range<Lsn>,
_reconstruct_state: &mut ValueReconstructState,
) -> Result<ValueReconstructResult> {
bail!(
"layer {} needs to be downloaded",
self.filename().file_name()
);
}
fn is_incremental(&self) -> bool {
self.is_incremental
}
/// debugging function to print out the contents of the layer
fn dump(&self, _verbose: bool) -> Result<()> {
println!(
"----- remote layer for ten {} tli {} keys {}-{} lsn {}-{} ----",
self.tenantid,
self.timelineid,
self.key_range.start,
self.key_range.end,
self.lsn_range.start,
self.lsn_range.end
);
Ok(())
}
fn short_id(&self) -> String {
self.filename().file_name()
}
}
impl PersistentLayer for RemoteLayer {
fn get_tenant_id(&self) -> TenantId {
self.tenantid
}
fn get_timeline_id(&self) -> TimelineId {
self.timelineid
}
fn filename(&self) -> LayerFileName {
if self.is_delta {
DeltaFileName {
key_range: self.key_range.clone(),
lsn_range: self.lsn_range.clone(),
}
.into()
} else {
ImageFileName {
key_range: self.key_range.clone(),
lsn: self.lsn_range.start,
}
.into()
}
}
fn local_path(&self) -> Option<PathBuf> {
None
}
fn iter(&self) -> Result<LayerIter<'_>> {
bail!("cannot iterate a remote layer");
}
fn key_iter(&self) -> Result<LayerKeyIter<'_>> {
bail!("cannot iterate a remote layer");
}
fn delete(&self) -> Result<()> {
Ok(())
}
fn downcast_remote_layer<'a>(self: Arc<Self>) -> Option<std::sync::Arc<RemoteLayer>> {
Some(self)
}
fn is_remote_layer(&self) -> bool {
true
}
fn file_size(&self) -> Option<u64> {
self.layer_metadata.file_size()
}
}
impl RemoteLayer {
pub fn new_img(
tenantid: TenantId,
timelineid: TimelineId,
fname: &ImageFileName,
layer_metadata: &LayerFileMetadata,
) -> RemoteLayer {
RemoteLayer {
tenantid,
timelineid,
key_range: fname.key_range.clone(),
lsn_range: fname.lsn..(fname.lsn + 1),
is_delta: false,
is_incremental: false,
file_name: fname.to_owned().into(),
layer_metadata: layer_metadata.clone(),
ongoing_download: Arc::new(tokio::sync::Semaphore::new(1)),
}
}
pub fn new_delta(
tenantid: TenantId,
timelineid: TimelineId,
fname: &DeltaFileName,
layer_metadata: &LayerFileMetadata,
) -> RemoteLayer {
RemoteLayer {
tenantid,
timelineid,
key_range: fname.key_range.clone(),
lsn_range: fname.lsn_range.clone(),
is_delta: true,
is_incremental: true,
file_name: fname.to_owned().into(),
layer_metadata: layer_metadata.clone(),
ongoing_download: Arc::new(tokio::sync::Semaphore::new(1)),
}
}
/// Create a Layer struct representing this layer, after it has been downloaded.
pub fn create_downloaded_layer(
&self,
conf: &'static PageServerConf,
file_size: u64,
) -> Arc<dyn PersistentLayer> {
if self.is_delta {
let fname = DeltaFileName {
key_range: self.key_range.clone(),
lsn_range: self.lsn_range.clone(),
};
Arc::new(DeltaLayer::new(
conf,
self.timelineid,
self.tenantid,
&fname,
file_size,
))
} else {
let fname = ImageFileName {
key_range: self.key_range.clone(),
lsn: self.lsn_range.start,
};
Arc::new(ImageLayer::new(
conf,
self.timelineid,
self.tenantid,
&fname,
file_size,
))
}
}
}

View File

@@ -97,8 +97,6 @@ pub(super) async fn gather_inputs(
// used to determine the `retention_period` for the size model
let mut max_cutoff_distance = None;
// this will probably conflict with on-demand downloaded layers, or at least force them all
// to be downloaded
for timeline in timelines {
let last_record_lsn = timeline.get_last_record_lsn();

View File

@@ -8,6 +8,7 @@ use anyhow::Result;
use bytes::Bytes;
use std::ops::Range;
use std::path::PathBuf;
use std::sync::Arc;
use utils::{
id::{TenantId, TimelineId},
@@ -15,6 +16,8 @@ use utils::{
};
use super::filename::LayerFileName;
use super::remote_layer::RemoteLayer;
pub fn range_overlaps<T>(a: &Range<T>, b: &Range<T>) -> bool
where
T: PartialOrd<T>,
@@ -161,4 +164,28 @@ pub trait PersistentLayer: Layer {
/// Permanently remove this layer from disk.
fn delete(&self) -> Result<()>;
fn downcast_remote_layer(self: Arc<Self>) -> Option<std::sync::Arc<RemoteLayer>> {
None
}
fn is_remote_layer(&self) -> bool {
false
}
/// Returns None if the layer file size is not known.
///
/// Should not change over the lifetime of the layer object because
/// current_physical_size is computed as the som of this value.
fn file_size(&self) -> Option<u64>;
}
pub fn downcast_remote_layer(
layer: &Arc<dyn PersistentLayer>,
) -> Option<std::sync::Arc<RemoteLayer>> {
if layer.is_remote_layer() {
Arc::clone(layer).downcast_remote_layer()
} else {
None
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -12,7 +12,7 @@
//!
use crate::metrics::{STORAGE_IO_SIZE, STORAGE_IO_TIME};
use once_cell::sync::OnceCell;
use std::fs::{File, OpenOptions};
use std::fs::{self, File, OpenOptions};
use std::io::{Error, ErrorKind, Read, Seek, SeekFrom, Write};
use std::os::unix::fs::FileExt;
use std::path::{Path, PathBuf};
@@ -240,6 +240,10 @@ impl VirtualFile {
self.with_file("fsync", |file| file.sync_all())?
}
pub fn metadata(&self) -> Result<fs::Metadata, Error> {
self.with_file("metadata", |file| file.metadata())?
}
/// Helper function that looks up the underlying File for this VirtualFile,
/// opening it and evicting some other File if necessary. It calls 'func'
/// with the physical File.

View File

@@ -31,7 +31,10 @@ use bytes::{Buf, Bytes, BytesMut};
use tracing::*;
use crate::pgdatadir_mapping::*;
use crate::tenant::PageReconstructResult;
use crate::tenant::Timeline;
use crate::try_no_ondemand_download;
use crate::try_page_reconstruct_result as try_prr;
use crate::walrecord::*;
use crate::ZERO_PAGE;
use pageserver_api::reltag::{RelTag, SlruKind};
@@ -52,10 +55,10 @@ pub struct WalIngest<'a> {
}
impl<'a> WalIngest<'a> {
pub fn new(timeline: &Timeline, startpoint: Lsn) -> Result<WalIngest> {
pub fn new(timeline: &Timeline, startpoint: Lsn) -> anyhow::Result<WalIngest> {
// Fetch the latest checkpoint into memory, so that we can compare with it
// quickly in `ingest_record` and update it when it changes.
let checkpoint_bytes = timeline.get_checkpoint(startpoint)?;
let checkpoint_bytes = timeline.get_checkpoint(startpoint).no_ondemand_download()?;
let checkpoint = CheckPoint::decode(&checkpoint_bytes)?;
trace!("CheckPoint.nextXid = {}", checkpoint.nextXid.value);
@@ -80,10 +83,12 @@ impl<'a> WalIngest<'a> {
lsn: Lsn,
modification: &mut DatadirModification,
decoded: &mut DecodedWALRecord,
) -> Result<()> {
) -> PageReconstructResult<()> {
modification.lsn = lsn;
decode_wal_record(recdata, decoded, self.timeline.pg_version)
.context("failed decoding wal record")?;
try_prr!(
decode_wal_record(recdata, decoded, self.timeline.pg_version)
.context("failed decoding wal record")
);
let mut buf = decoded.record.clone();
buf.advance(decoded.main_data_offset);
@@ -98,7 +103,7 @@ impl<'a> WalIngest<'a> {
if decoded.xl_rmid == pg_constants::RM_HEAP_ID
|| decoded.xl_rmid == pg_constants::RM_HEAP2_ID
{
self.ingest_heapam_record(&mut buf, modification, decoded)?;
try_prr!(self.ingest_heapam_record(&mut buf, modification, decoded));
}
// Handle other special record types
if decoded.xl_rmid == pg_constants::RM_SMGR_ID
@@ -106,13 +111,13 @@ impl<'a> WalIngest<'a> {
== pg_constants::XLOG_SMGR_CREATE
{
let create = XlSmgrCreate::decode(&mut buf);
self.ingest_xlog_smgr_create(modification, &create)?;
try_prr!(self.ingest_xlog_smgr_create(modification, &create));
} else if decoded.xl_rmid == pg_constants::RM_SMGR_ID
&& (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK)
== pg_constants::XLOG_SMGR_TRUNCATE
{
let truncate = XlSmgrTruncate::decode(&mut buf);
self.ingest_xlog_smgr_truncate(modification, &truncate)?;
try_prr!(self.ingest_xlog_smgr_truncate(modification, &truncate));
} else if decoded.xl_rmid == pg_constants::RM_DBASE_ID {
debug!(
"handle RM_DBASE_ID for Postgres version {:?}",
@@ -125,14 +130,14 @@ impl<'a> WalIngest<'a> {
let createdb = XlCreateDatabase::decode(&mut buf);
debug!("XLOG_DBASE_CREATE v14");
self.ingest_xlog_dbase_create(modification, &createdb)?;
try_prr!(self.ingest_xlog_dbase_create(modification, &createdb));
} else if (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK)
== postgres_ffi::v14::bindings::XLOG_DBASE_DROP
{
let dropdb = XlDropDatabase::decode(&mut buf);
for tablespace_id in dropdb.tablespace_ids {
trace!("Drop db {}, {}", tablespace_id, dropdb.db_id);
modification.drop_dbdir(tablespace_id, dropdb.db_id)?;
try_prr!(modification.drop_dbdir(tablespace_id, dropdb.db_id));
}
}
} else if self.timeline.pg_version == 15 {
@@ -148,14 +153,14 @@ impl<'a> WalIngest<'a> {
// So we can reuse XlCreateDatabase here.
debug!("XLOG_DBASE_CREATE_FILE_COPY");
let createdb = XlCreateDatabase::decode(&mut buf);
self.ingest_xlog_dbase_create(modification, &createdb)?;
try_prr!(self.ingest_xlog_dbase_create(modification, &createdb));
} else if (decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK)
== postgres_ffi::v15::bindings::XLOG_DBASE_DROP
{
let dropdb = XlDropDatabase::decode(&mut buf);
for tablespace_id in dropdb.tablespace_ids {
trace!("Drop db {}, {}", tablespace_id, dropdb.db_id);
modification.drop_dbdir(tablespace_id, dropdb.db_id)?;
try_prr!(modification.drop_dbdir(tablespace_id, dropdb.db_id));
}
}
}
@@ -167,38 +172,38 @@ impl<'a> WalIngest<'a> {
let pageno = buf.get_u32_le();
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
self.put_slru_page_image(
try_prr!(self.put_slru_page_image(
modification,
SlruKind::Clog,
segno,
rpageno,
ZERO_PAGE.clone(),
)?;
));
} else {
assert!(info == pg_constants::CLOG_TRUNCATE);
let xlrec = XlClogTruncate::decode(&mut buf);
self.ingest_clog_truncate_record(modification, &xlrec)?;
try_prr!(self.ingest_clog_truncate_record(modification, &xlrec));
}
} else if decoded.xl_rmid == pg_constants::RM_XACT_ID {
let info = decoded.xl_info & pg_constants::XLOG_XACT_OPMASK;
if info == pg_constants::XLOG_XACT_COMMIT || info == pg_constants::XLOG_XACT_ABORT {
let parsed_xact =
XlXactParsedRecord::decode(&mut buf, decoded.xl_xid, decoded.xl_info);
self.ingest_xact_record(
try_prr!(self.ingest_xact_record(
modification,
&parsed_xact,
info == pg_constants::XLOG_XACT_COMMIT,
)?;
));
} else if info == pg_constants::XLOG_XACT_COMMIT_PREPARED
|| info == pg_constants::XLOG_XACT_ABORT_PREPARED
{
let parsed_xact =
XlXactParsedRecord::decode(&mut buf, decoded.xl_xid, decoded.xl_info);
self.ingest_xact_record(
try_prr!(self.ingest_xact_record(
modification,
&parsed_xact,
info == pg_constants::XLOG_XACT_COMMIT_PREPARED,
)?;
));
// Remove twophase file. see RemoveTwoPhaseFile() in postgres code
trace!(
"Drop twophaseFile for xid {} parsed_xact.xid {} here at {}",
@@ -206,9 +211,10 @@ impl<'a> WalIngest<'a> {
parsed_xact.xid,
lsn,
);
modification.drop_twophase_file(parsed_xact.xid)?;
try_prr!(modification.drop_twophase_file(parsed_xact.xid));
} else if info == pg_constants::XLOG_XACT_PREPARE {
modification.put_twophase_file(decoded.xl_xid, Bytes::copy_from_slice(&buf[..]))?;
try_prr!(modification
.put_twophase_file(decoded.xl_xid, Bytes::copy_from_slice(&buf[..])));
}
} else if decoded.xl_rmid == pg_constants::RM_MULTIXACT_ID {
let info = decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK;
@@ -217,34 +223,34 @@ impl<'a> WalIngest<'a> {
let pageno = buf.get_u32_le();
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
self.put_slru_page_image(
try_prr!(self.put_slru_page_image(
modification,
SlruKind::MultiXactOffsets,
segno,
rpageno,
ZERO_PAGE.clone(),
)?;
));
} else if info == pg_constants::XLOG_MULTIXACT_ZERO_MEM_PAGE {
let pageno = buf.get_u32_le();
let segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
let rpageno = pageno % pg_constants::SLRU_PAGES_PER_SEGMENT;
self.put_slru_page_image(
try_prr!(self.put_slru_page_image(
modification,
SlruKind::MultiXactMembers,
segno,
rpageno,
ZERO_PAGE.clone(),
)?;
));
} else if info == pg_constants::XLOG_MULTIXACT_CREATE_ID {
let xlrec = XlMultiXactCreate::decode(&mut buf);
self.ingest_multixact_create_record(modification, &xlrec)?;
try_prr!(self.ingest_multixact_create_record(modification, &xlrec));
} else if info == pg_constants::XLOG_MULTIXACT_TRUNCATE_ID {
let xlrec = XlMultiXactTruncate::decode(&mut buf);
self.ingest_multixact_truncate_record(modification, &xlrec)?;
try_prr!(self.ingest_multixact_truncate_record(modification, &xlrec));
}
} else if decoded.xl_rmid == pg_constants::RM_RELMAP_ID {
let xlrec = XlRelmapUpdate::decode(&mut buf);
self.ingest_relmap_page(modification, &xlrec, decoded)?;
try_prr!(self.ingest_relmap_page(modification, &xlrec, decoded));
} else if decoded.xl_rmid == pg_constants::RM_XLOG_ID {
let info = decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK;
if info == pg_constants::XLOG_NEXTOID {
@@ -258,7 +264,9 @@ impl<'a> WalIngest<'a> {
{
let mut checkpoint_bytes = [0u8; SIZEOF_CHECKPOINT];
buf.copy_to_slice(&mut checkpoint_bytes);
let xlog_checkpoint = CheckPoint::decode(&checkpoint_bytes)?;
let xlog_checkpoint = try_prr!(
CheckPoint::decode(&checkpoint_bytes).context("deserialize CheckPoint")
);
trace!(
"xlog_checkpoint.oldestXid={}, checkpoint.oldestXid={}",
xlog_checkpoint.oldestXid,
@@ -279,22 +287,23 @@ impl<'a> WalIngest<'a> {
// Iterate through all the blocks that the record modifies, and
// "put" a separate copy of the record for each block.
for blk in decoded.blocks.iter() {
self.ingest_decoded_block(modification, lsn, decoded, blk)?;
try_no_ondemand_download!(self.ingest_decoded_block(modification, lsn, decoded, blk));
}
// If checkpoint data was updated, store the new version in the repository
if self.checkpoint_modified {
let new_checkpoint_bytes = self.checkpoint.encode()?;
let new_checkpoint_bytes =
try_prr!(self.checkpoint.encode().context("encode checkpoint"));
modification.put_checkpoint(new_checkpoint_bytes)?;
try_prr!(modification.put_checkpoint(new_checkpoint_bytes));
self.checkpoint_modified = false;
}
// Now that this record has been fully handled, including updating the
// checkpoint data, let the repository know that it is up-to-date to this LSN
modification.commit()?;
try_prr!(modification.commit());
Ok(())
PageReconstructResult::Success(())
}
fn ingest_decoded_block(
@@ -303,7 +312,7 @@ impl<'a> WalIngest<'a> {
lsn: Lsn,
decoded: &DecodedWALRecord,
blk: &DecodedBkpBlock,
) -> Result<()> {
) -> PageReconstructResult<()> {
let rel = RelTag {
spcnode: blk.rnode_spcnode,
dbnode: blk.rnode_dbnode,
@@ -323,7 +332,7 @@ impl<'a> WalIngest<'a> {
&& (decoded.xl_info == pg_constants::XLOG_FPI
|| decoded.xl_info == pg_constants::XLOG_FPI_FOR_HINT)
// compression of WAL is not yet supported: fall back to storing the original WAL record
&& !postgres_ffi::bkpimage_is_compressed(blk.bimg_info, self.timeline.pg_version)?
&& !try_prr!(postgres_ffi::bkpimage_is_compressed(blk.bimg_info, self.timeline.pg_version))
{
// Extract page image from FPI record
let img_len = blk.bimg_len as usize;
@@ -345,15 +354,20 @@ impl<'a> WalIngest<'a> {
page_set_lsn(&mut image, lsn)
}
assert_eq!(image.len(), BLCKSZ as usize);
self.put_rel_page_image(modification, rel, blk.blkno, image.freeze())?;
try_no_ondemand_download!(self.put_rel_page_image(
modification,
rel,
blk.blkno,
image.freeze()
));
} else {
let rec = NeonWalRecord::Postgres {
will_init: blk.will_init || blk.apply_image,
rec: decoded.record.clone(),
};
self.put_rel_wal_record(modification, rel, blk.blkno, rec)?;
try_prr!(self.put_rel_wal_record(modification, rel, blk.blkno, rec));
}
Ok(())
PageReconstructResult::Success(())
}
fn ingest_heapam_record(
@@ -505,7 +519,7 @@ impl<'a> WalIngest<'a> {
&mut self,
modification: &mut DatadirModification,
rec: &XlCreateDatabase,
) -> Result<()> {
) -> anyhow::Result<()> {
let db_id = rec.db_id;
let tablespace_id = rec.tablespace_id;
let src_db_id = rec.src_db_id;
@@ -520,14 +534,16 @@ impl<'a> WalIngest<'a> {
let rels = modification
.tline
.list_rels(src_tablespace_id, src_db_id, req_lsn)?;
.list_rels(src_tablespace_id, src_db_id, req_lsn)
.no_ondemand_download()?;
debug!("ingest_xlog_dbase_create: {} rels", rels.len());
// Copy relfilemap
let filemap = modification
.tline
.get_relmap_file(src_tablespace_id, src_db_id, req_lsn)?;
.get_relmap_file(src_tablespace_id, src_db_id, req_lsn)
.no_ondemand_download()?;
modification.put_relmap_file(tablespace_id, db_id, filemap)?;
let mut num_rels_copied = 0;
@@ -536,7 +552,10 @@ impl<'a> WalIngest<'a> {
assert_eq!(src_rel.spcnode, src_tablespace_id);
assert_eq!(src_rel.dbnode, src_db_id);
let nblocks = modification.tline.get_rel_size(src_rel, req_lsn, true)?;
let nblocks = modification
.tline
.get_rel_size(src_rel, req_lsn, true)
.no_ondemand_download()?;
let dst_rel = RelTag {
spcnode: tablespace_id,
dbnode: db_id,
@@ -553,7 +572,8 @@ impl<'a> WalIngest<'a> {
let content = modification
.tline
.get_rel_page_at_lsn(src_rel, blknum, req_lsn, true)?;
.get_rel_page_at_lsn(src_rel, blknum, req_lsn, true)
.no_ondemand_download()?;
modification.put_rel_page_image(dst_rel, blknum, content)?;
num_blocks_copied += 1;
}
@@ -657,7 +677,7 @@ impl<'a> WalIngest<'a> {
modification: &mut DatadirModification,
parsed: &XlXactParsedRecord,
is_commit: bool,
) -> Result<()> {
) -> anyhow::Result<()> {
// Record update of CLOG pages
let mut pageno = parsed.xid / pg_constants::CLOG_XACTS_PER_PAGE;
let mut segno = pageno / pg_constants::SLRU_PAGES_PER_SEGMENT;
@@ -713,7 +733,11 @@ impl<'a> WalIngest<'a> {
relnode: xnode.relnode,
};
let last_lsn = self.timeline.get_last_record_lsn();
if modification.tline.get_rel_exists(rel, last_lsn, true)? {
if modification
.tline
.get_rel_exists(rel, last_lsn, true)
.no_ondemand_download()?
{
self.put_rel_drop(modification, rel)?;
}
}
@@ -725,7 +749,7 @@ impl<'a> WalIngest<'a> {
&mut self,
modification: &mut DatadirModification,
xlrec: &XlClogTruncate,
) -> Result<()> {
) -> anyhow::Result<()> {
info!(
"RM_CLOG_ID truncate pageno {} oldestXid {} oldestXidDB {}",
xlrec.pageno, xlrec.oldest_xid, xlrec.oldest_xid_db
@@ -767,7 +791,8 @@ impl<'a> WalIngest<'a> {
let req_lsn = modification.tline.get_last_record_lsn();
for segno in modification
.tline
.list_slru_segments(SlruKind::Clog, req_lsn)?
.list_slru_segments(SlruKind::Clog, req_lsn)
.no_ondemand_download()?
{
let segpage = segno * pg_constants::SLRU_PAGES_PER_SEGMENT;
if slru_may_delete_clogsegment(segpage, xlrec.pageno) {
@@ -923,10 +948,10 @@ impl<'a> WalIngest<'a> {
rel: RelTag,
blknum: BlockNumber,
img: Bytes,
) -> Result<()> {
self.handle_rel_extend(modification, rel, blknum)?;
modification.put_rel_page_image(rel, blknum, img)?;
Ok(())
) -> PageReconstructResult<()> {
try_no_ondemand_download!(self.handle_rel_extend(modification, rel, blknum));
try_prr!(modification.put_rel_page_image(rel, blknum, img));
PageReconstructResult::Success(())
}
fn put_rel_wal_record(
@@ -936,7 +961,8 @@ impl<'a> WalIngest<'a> {
blknum: BlockNumber,
rec: NeonWalRecord,
) -> Result<()> {
self.handle_rel_extend(modification, rel, blknum)?;
self.handle_rel_extend(modification, rel, blknum)
.no_ondemand_download()?;
modification.put_rel_wal_record(rel, blknum, rec)?;
Ok(())
}
@@ -946,7 +972,7 @@ impl<'a> WalIngest<'a> {
modification: &mut DatadirModification,
rel: RelTag,
nblocks: BlockNumber,
) -> Result<()> {
) -> anyhow::Result<()> {
modification.put_rel_truncation(rel, nblocks)?;
Ok(())
}
@@ -956,11 +982,17 @@ impl<'a> WalIngest<'a> {
Ok(())
}
fn get_relsize(&mut self, rel: RelTag, lsn: Lsn) -> Result<BlockNumber> {
let nblocks = if !self.timeline.get_rel_exists(rel, lsn, true)? {
fn get_relsize(&mut self, rel: RelTag, lsn: Lsn) -> anyhow::Result<BlockNumber> {
let nblocks = if !self
.timeline
.get_rel_exists(rel, lsn, true)
.no_ondemand_download()?
{
0
} else {
self.timeline.get_rel_size(rel, lsn, true)?
self.timeline
.get_rel_size(rel, lsn, true)
.no_ondemand_download()?
};
Ok(nblocks)
}
@@ -970,30 +1002,31 @@ impl<'a> WalIngest<'a> {
modification: &mut DatadirModification,
rel: RelTag,
blknum: BlockNumber,
) -> Result<()> {
) -> PageReconstructResult<()> {
let new_nblocks = blknum + 1;
// Check if the relation exists. We implicitly create relations on first
// record.
// TODO: would be nice if to be more explicit about it
let last_lsn = modification.lsn;
let old_nblocks = if !self.timeline.get_rel_exists(rel, last_lsn, true)? {
// create it with 0 size initially, the logic below will extend it
modification.put_rel_creation(rel, 0)?;
0
} else {
self.timeline.get_rel_size(rel, last_lsn, true)?
};
let old_nblocks =
if !try_no_ondemand_download!(self.timeline.get_rel_exists(rel, last_lsn, true)) {
// create it with 0 size initially, the logic below will extend it
try_prr!(modification.put_rel_creation(rel, 0));
0
} else {
try_no_ondemand_download!(self.timeline.get_rel_size(rel, last_lsn, true))
};
if new_nblocks > old_nblocks {
//info!("extending {} {} to {}", rel, old_nblocks, new_nblocks);
modification.put_rel_extend(rel, new_nblocks)?;
try_prr!(modification.put_rel_extend(rel, new_nblocks));
// fill the gap with zeros
for gap_blknum in old_nblocks..blknum {
modification.put_rel_page_image(rel, gap_blknum, ZERO_PAGE.clone())?;
try_prr!(modification.put_rel_page_image(rel, gap_blknum, ZERO_PAGE.clone()));
}
}
Ok(())
PageReconstructResult::Success(())
}
fn put_slru_page_image(
@@ -1015,7 +1048,7 @@ impl<'a> WalIngest<'a> {
kind: SlruKind,
segno: u32,
blknum: BlockNumber,
) -> Result<()> {
) -> anyhow::Result<()> {
// we don't use a cache for this like we do for relations. SLRUS are explcitly
// extended with ZEROPAGE records, not with commit records, so it happens
// a lot less frequently.
@@ -1027,13 +1060,16 @@ impl<'a> WalIngest<'a> {
let last_lsn = self.timeline.get_last_record_lsn();
let old_nblocks = if !self
.timeline
.get_slru_segment_exists(kind, segno, last_lsn)?
.get_slru_segment_exists(kind, segno, last_lsn)
.no_ondemand_download()?
{
// create it with 0 size initially, the logic below will extend it
modification.put_slru_segment_creation(kind, segno, 0)?;
0
} else {
self.timeline.get_slru_segment_size(kind, segno, last_lsn)?
self.timeline
.get_slru_segment_size(kind, segno, last_lsn)
.no_ondemand_download()?
};
if new_nblocks > old_nblocks {
@@ -1099,58 +1135,103 @@ mod tests {
let mut m = tline.begin_modification(Lsn(0x20));
walingest.put_rel_creation(&mut m, TESTREL_A)?;
walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))
.no_ondemand_download()?;
m.commit()?;
let mut m = tline.begin_modification(Lsn(0x30));
walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 3"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 3"))
.no_ondemand_download()?;
m.commit()?;
let mut m = tline.begin_modification(Lsn(0x40));
walingest.put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1 at 4"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1 at 4"))
.no_ondemand_download()?;
m.commit()?;
let mut m = tline.begin_modification(Lsn(0x50));
walingest.put_rel_page_image(&mut m, TESTREL_A, 2, TEST_IMG("foo blk 2 at 5"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 2, TEST_IMG("foo blk 2 at 5"))
.no_ondemand_download()?;
m.commit()?;
assert_current_logical_size(&*tline, Lsn(0x50));
// The relation was created at LSN 2, not visible at LSN 1 yet.
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x10), false)?, false);
assert!(tline.get_rel_size(TESTREL_A, Lsn(0x10), false).is_err());
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x10), false)
.no_ondemand_download()?,
false
);
assert!(tline
.get_rel_size(TESTREL_A, Lsn(0x10), false)
.no_ondemand_download()
.is_err());
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20), false)?, true);
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20), false)?, 1);
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50), false)?, 3);
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x20), false)
.no_ondemand_download()?,
true
);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x20), false)
.no_ondemand_download()?,
1
);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x50), false)
.no_ondemand_download()?,
3
);
// Check page contents at each LSN
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x20), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x20), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 0 at 2")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x30), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x30), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 0 at 3")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x40), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x40), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 0 at 3")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 1 at 4")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x50), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x50), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 0 at 3")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 1 at 4")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 2 at 5")
);
@@ -1161,20 +1242,36 @@ mod tests {
assert_current_logical_size(&*tline, Lsn(0x60));
// Check reported size and contents after truncation
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x60), false)?, 2);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x60), false)?,
tline
.get_rel_size(TESTREL_A, Lsn(0x60), false)
.no_ondemand_download()?,
2
);
assert_eq!(
tline
.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x60), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 0 at 3")
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 1 at 4")
);
// should still see the truncated block with older LSN
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50), false)?, 3);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false)?,
tline
.get_rel_size(TESTREL_A, Lsn(0x50), false)
.no_ondemand_download()?,
3
);
assert_eq!(
tline
.get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 2 at 5")
);
@@ -1182,35 +1279,62 @@ mod tests {
let mut m = tline.begin_modification(Lsn(0x68));
walingest.put_rel_truncation(&mut m, TESTREL_A, 0)?;
m.commit()?;
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x68), false)?, 0);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x68), false)
.no_ondemand_download()?,
0
);
// Extend from 0 to 2 blocks, leaving a gap
let mut m = tline.begin_modification(Lsn(0x70));
walingest.put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 1, TEST_IMG("foo blk 1"))
.no_ondemand_download()?;
m.commit()?;
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x70), false)?, 2);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x70), false)?,
tline
.get_rel_size(TESTREL_A, Lsn(0x70), false)
.no_ondemand_download()?,
2
);
assert_eq!(
tline
.get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x70), false)
.no_ondemand_download()?,
ZERO_PAGE
);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x70), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x70), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 1")
);
// Extend a lot more, leaving a big gap that spans across segments
let mut m = tline.begin_modification(Lsn(0x80));
walingest.put_rel_page_image(&mut m, TESTREL_A, 1500, TEST_IMG("foo blk 1500"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 1500, TEST_IMG("foo blk 1500"))
.no_ondemand_download()?;
m.commit()?;
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x80), false)?, 1501);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x80), false)
.no_ondemand_download()?,
1501
);
for blk in 2..1500 {
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, blk, Lsn(0x80), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, blk, Lsn(0x80), false)
.no_ondemand_download()?,
ZERO_PAGE
);
}
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, 1500, Lsn(0x80), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, 1500, Lsn(0x80), false)
.no_ondemand_download()?,
TEST_IMG("foo blk 1500")
);
@@ -1226,12 +1350,24 @@ mod tests {
let mut walingest = init_walingest_test(&*tline)?;
let mut m = tline.begin_modification(Lsn(0x20));
walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 2"))
.no_ondemand_download()?;
m.commit()?;
// Check that rel exists and size is correct
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20), false)?, true);
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20), false)?, 1);
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x20), false)
.no_ondemand_download()?,
true
);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x20), false)
.no_ondemand_download()?,
1
);
// Drop rel
let mut m = tline.begin_modification(Lsn(0x30));
@@ -1239,19 +1375,36 @@ mod tests {
m.commit()?;
// Check that rel is not visible anymore
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x30), false)?, false);
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x30), false)
.no_ondemand_download()?,
false
);
// FIXME: should fail
//assert!(tline.get_rel_size(TESTREL_A, Lsn(0x30), false)?.is_none());
// Re-create it
let mut m = tline.begin_modification(Lsn(0x40));
walingest.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 4"))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, 0, TEST_IMG("foo blk 0 at 4"))
.no_ondemand_download()?;
m.commit()?;
// Check that rel exists and size is correct
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x40), false)?, true);
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x40), false)?, 1);
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x40), false)
.no_ondemand_download()?,
true
);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x40), false)
.no_ondemand_download()?,
1
);
Ok(())
}
@@ -1270,23 +1423,45 @@ mod tests {
let mut m = tline.begin_modification(Lsn(0x20));
for blkno in 0..relsize {
let data = format!("foo blk {} at {}", blkno, Lsn(0x20));
walingest.put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data))
.no_ondemand_download()?;
}
m.commit()?;
// The relation was created at LSN 20, not visible at LSN 1 yet.
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x10), false)?, false);
assert!(tline.get_rel_size(TESTREL_A, Lsn(0x10), false).is_err());
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x10), false)
.no_ondemand_download()?,
false
);
assert!(tline
.get_rel_size(TESTREL_A, Lsn(0x10), false)
.no_ondemand_download()
.is_err());
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x20), false)?, true);
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x20), false)?, relsize);
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x20), false)
.no_ondemand_download()?,
true
);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x20), false)
.no_ondemand_download()?,
relsize
);
// Check relation content
for blkno in 0..relsize {
let lsn = Lsn(0x20);
let data = format!("foo blk {} at {}", blkno, lsn);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, blkno, lsn, false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, blkno, lsn, false)
.no_ondemand_download()?,
TEST_IMG(&data)
);
}
@@ -1298,24 +1473,38 @@ mod tests {
m.commit()?;
// Check reported size and contents after truncation
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x60), false)?, 1);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x60), false)
.no_ondemand_download()?,
1
);
for blkno in 0..1 {
let lsn = Lsn(0x20);
let data = format!("foo blk {} at {}", blkno, lsn);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x60), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x60), false)
.no_ondemand_download()?,
TEST_IMG(&data)
);
}
// should still see all blocks with older LSN
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x50), false)?, relsize);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x50), false)
.no_ondemand_download()?,
relsize
);
for blkno in 0..relsize {
let lsn = Lsn(0x20);
let data = format!("foo blk {} at {}", blkno, lsn);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x50), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x50), false)
.no_ondemand_download()?,
TEST_IMG(&data)
);
}
@@ -1326,18 +1515,32 @@ mod tests {
let mut m = tline.begin_modification(lsn);
for blkno in 0..relsize {
let data = format!("foo blk {} at {}", blkno, lsn);
walingest.put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data))?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, blkno, TEST_IMG(&data))
.no_ondemand_download()?;
}
m.commit()?;
assert_eq!(tline.get_rel_exists(TESTREL_A, Lsn(0x80), false)?, true);
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(0x80), false)?, relsize);
assert_eq!(
tline
.get_rel_exists(TESTREL_A, Lsn(0x80), false)
.no_ondemand_download()?,
true
);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(0x80), false)
.no_ondemand_download()?,
relsize
);
// Check relation content
for blkno in 0..relsize {
let lsn = Lsn(0x80);
let data = format!("foo blk {} at {}", blkno, lsn);
assert_eq!(
tline.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x80), false)?,
tline
.get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x80), false)
.no_ondemand_download()?,
TEST_IMG(&data)
);
}
@@ -1358,14 +1561,18 @@ mod tests {
lsn += 0x10;
let mut m = tline.begin_modification(Lsn(lsn));
let img = TEST_IMG(&format!("foo blk {} at {}", blknum, Lsn(lsn)));
walingest.put_rel_page_image(&mut m, TESTREL_A, blknum as BlockNumber, img)?;
walingest
.put_rel_page_image(&mut m, TESTREL_A, blknum as BlockNumber, img)
.no_ondemand_download()?;
m.commit()?;
}
assert_current_logical_size(&*tline, Lsn(lsn));
assert_eq!(
tline.get_rel_size(TESTREL_A, Lsn(lsn), false)?,
tline
.get_rel_size(TESTREL_A, Lsn(lsn), false)
.no_ondemand_download()?,
RELSEG_SIZE + 1
);
@@ -1374,7 +1581,12 @@ mod tests {
let mut m = tline.begin_modification(Lsn(lsn));
walingest.put_rel_truncation(&mut m, TESTREL_A, RELSEG_SIZE)?;
m.commit()?;
assert_eq!(tline.get_rel_size(TESTREL_A, Lsn(lsn), false)?, RELSEG_SIZE);
assert_eq!(
tline
.get_rel_size(TESTREL_A, Lsn(lsn), false)
.no_ondemand_download()?,
RELSEG_SIZE
);
assert_current_logical_size(&*tline, Lsn(lsn));
// Truncate another block
@@ -1383,7 +1595,9 @@ mod tests {
walingest.put_rel_truncation(&mut m, TESTREL_A, RELSEG_SIZE - 1)?;
m.commit()?;
assert_eq!(
tline.get_rel_size(TESTREL_A, Lsn(lsn), false)?,
tline
.get_rel_size(TESTREL_A, Lsn(lsn), false)
.no_ondemand_download()?,
RELSEG_SIZE - 1
);
assert_current_logical_size(&*tline, Lsn(lsn));
@@ -1397,7 +1611,9 @@ mod tests {
walingest.put_rel_truncation(&mut m, TESTREL_A, size as BlockNumber)?;
m.commit()?;
assert_eq!(
tline.get_rel_size(TESTREL_A, Lsn(lsn), false)?,
tline
.get_rel_size(TESTREL_A, Lsn(lsn), false)
.no_ondemand_download()?,
size as BlockNumber
);

View File

@@ -407,7 +407,7 @@ impl WalreceiverState {
.await
.context("walreceiver connection handling failure")
}
.instrument(info_span!("walreceiver_connection", id = %id))
.instrument(info_span!("walreceiver_connection", id = %id, node_id = %new_sk_id))
});
let now = Utc::now().naive_utc();

View File

@@ -20,7 +20,9 @@ use tokio::{pin, select, sync::watch, time};
use tokio_postgres::{replication::ReplicationStream, Client};
use tracing::{debug, error, info, trace, warn};
use crate::{metrics::LIVE_CONNECTIONS_COUNT, walreceiver::TaskStateUpdate};
use crate::{
metrics::LIVE_CONNECTIONS_COUNT, tenant::with_ondemand_download, walreceiver::TaskStateUpdate,
};
use crate::{
task_mgr,
task_mgr::TaskKind,
@@ -248,9 +250,16 @@ pub async fn handle_walreceiver_connection(
// at risk of hitting a deadlock.
ensure!(lsn.is_aligned());
walingest
.ingest_record(recdata, lsn, &mut modification, &mut decoded)
.context("could not ingest record at {lsn}")?;
with_ondemand_download(|| {
walingest.ingest_record(
recdata.clone(),
lsn,
&mut modification,
&mut decoded,
)
})
.await
.with_context(|| format!("could not ingest record at {lsn}"))?;
fail_point!("walreceiver-after-ingest");

View File

@@ -1,6 +1,7 @@
//!
//! Functions for parsing WAL records.
//!
use anyhow::Result;
use bytes::{Buf, Bytes};
use postgres_ffi::pg_constants;