Compare commits

...

17 Commits

Author SHA1 Message Date
John Spray
6e53637b06 neon_local: pin pageservers and endpoints 2024-01-03 16:14:58 +00:00
John Spray
45bd85c372 pageserver: skip the blk range check: this is slow (reads relsize page) if queries aren't
hitting latest=true
2023-12-20 20:06:56 +00:00
John Spray
403aff42d7 timeline: skip materialized page cache for relation blocks 2023-12-20 20:06:46 +00:00
John Spray
fa113f8d40 page_cache: don't use a histogram (too expensive) 2023-12-20 20:05:06 +00:00
John Spray
8ec1b57a2c pagebench: hack around bug 2023-12-20 20:03:23 +00:00
John Spray
a8ec7d7ad8 pageserver: prototype of skipping page cache for non-index block reads 2023-12-20 19:18:26 +00:00
Christian Schwarz
5f7e821a62 make CI happy 2023-12-20 15:53:21 +00:00
Christian Schwarz
c417a23dd0 pagebench: factor out the concept of thread local stats 2023-12-18 18:32:22 +00:00
Christian Schwarz
20e5e9dd16 pagebench: finish trigger initial logical size calculation benchmark 2023-12-18 18:32:22 +00:00
Christian Schwarz
24c72db5ff pagebench: centralize target discovery 2023-12-18 18:32:22 +00:00
Christian Schwarz
6aee8511f7 pagebench: getpage: WIP: when auto-discovering timelines, add ability to limit 2023-12-18 18:32:22 +00:00
Christian Schwarz
ad2091bdd0 pagebench: WIP: command to trigger initial logical size calculation 2023-12-18 18:32:21 +00:00
Christian Schwarz
573d4752e6 pagebench: add a 'getpage@lsn' benchmark 2023-12-18 18:32:21 +00:00
Christian Schwarz
136bec6014 pagebench: add a 'basebackup' benchmark 2023-12-18 18:32:21 +00:00
Christian Schwarz
0f8b4faa50 pagebench: scaffold 2023-12-18 18:32:21 +00:00
Christian Schwarz
5b42949531 Merge branch 'main' into problame/benchmarking/pr/timeline-ids-in-tenant-details 2023-12-18 19:22:19 +01:00
Christian Schwarz
4a6dfb0ccb include timeline ids in tenant details response 2023-12-18 15:12:48 +00:00
31 changed files with 1260 additions and 46 deletions

36
Cargo.lock generated
View File

@@ -2106,6 +2106,20 @@ dependencies = [
"hashbrown 0.13.2",
]
[[package]]
name = "hdrhistogram"
version = "7.5.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "765c9198f173dd59ce26ff9f95ef0aafd0a0fe01fb9d72841bc5066a4c06511d"
dependencies = [
"base64 0.21.1",
"byteorder",
"crossbeam-channel",
"flate2",
"nom",
"num-traits",
]
[[package]]
name = "heapless"
version = "0.8.0"
@@ -3056,6 +3070,28 @@ dependencies = [
"sha2",
]
[[package]]
name = "pagebench"
version = "0.1.0"
dependencies = [
"anyhow",
"clap",
"futures",
"hdrhistogram",
"humantime",
"humantime-serde",
"pageserver",
"pageserver_api",
"pageserver_client",
"rand 0.8.5",
"serde",
"serde_json",
"tokio",
"tracing",
"utils",
"workspace_hack",
]
[[package]]
name = "pagectl"
version = "0.1.0"

View File

@@ -6,6 +6,7 @@ members = [
"pageserver",
"pageserver/ctl",
"pageserver/client",
"pageserver/pagebench",
"proxy",
"safekeeper",
"storage_broker",
@@ -79,6 +80,7 @@ futures-util = "0.3"
git-version = "0.3"
hashbrown = "0.13"
hashlink = "0.8.1"
hdrhistogram = "7.5.2"
hex = "0.4"
hex-literal = "0.4"
hmac = "0.12.1"

View File

@@ -549,8 +549,10 @@ impl Endpoint {
// Launch compute_ctl
println!("Starting postgres node at '{}'", self.connstr());
let mut cmd = Command::new(self.env.neon_distrib_dir.join("compute_ctl"));
cmd.args(["--http-port", &self.http_address.port().to_string()])
let mut cmd = Command::new("/usr/bin/taskset");
cmd.args(["-c".to_string(), "8-11".to_string()])
.args([self.env.neon_distrib_dir.join("compute_ctl")])
.args(["--http-port", &self.http_address.port().to_string()])
.args(["--pgdata", self.pgdata().to_str().unwrap()])
.args(["--connstr", &self.connstr()])
.args([

View File

@@ -12,6 +12,7 @@ use std::io::Write;
use std::num::NonZeroU64;
use std::path::PathBuf;
use std::process::{Child, Command};
use std::str::FromStr;
use std::time::Duration;
use anyhow::{bail, Context};
@@ -216,11 +217,19 @@ impl PageServerNode {
if update_config {
args.push(Cow::Borrowed("--update-config"));
}
let mut taskset_args = vec![
"-c".to_string(),
format!("{}", self.conf.id.0 - 1),
self.env.pageserver_bin().to_string_lossy().into(),
];
taskset_args.extend(args.into_iter().map(|a| a.to_string()));
background_process::start_process(
"pageserver",
&datadir,
&self.env.pageserver_bin(),
args.iter().map(Cow::as_ref),
&PathBuf::from_str("/usr/bin/taskset").unwrap(),
taskset_args,
self.pageserver_env_variables()?,
background_process::InitialPidFile::Expect(self.pid_file()),
|| async {

View File

@@ -370,6 +370,14 @@ pub struct TenantInfo {
pub attachment_status: TenantAttachmentStatus,
}
#[derive(Serialize, Deserialize, Clone)]
pub struct TenantDetails {
#[serde(flatten)]
pub tenant_info: TenantInfo,
pub timelines: Vec<TimelineId>,
}
/// This represents the output of the "timeline_detail" and "timeline_list" API calls.
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct TimelineInfo {

View File

@@ -81,6 +81,10 @@ impl TenantShardId {
pub fn is_zero(&self) -> bool {
self.shard_number == ShardNumber(0)
}
pub fn is_unsharded(&self) -> bool {
self.shard_number == ShardNumber(0) && self.shard_count == ShardCount(0)
}
}
/// Formatting helper

View File

@@ -366,6 +366,47 @@ impl MonotonicCounter<Lsn> for RecordLsn {
}
}
/// Implements [`rand::distributions::uniform::UniformSampler`] so we can sample [`Lsn`]s.
pub struct LsnSampler(<u64 as rand::distributions::uniform::SampleUniform>::Sampler);
impl rand::distributions::uniform::SampleUniform for Lsn {
type Sampler = LsnSampler;
}
impl rand::distributions::uniform::UniformSampler for LsnSampler {
type X = Lsn;
fn new<B1, B2>(low: B1, high: B2) -> Self
where
B1: rand::distributions::uniform::SampleBorrow<Self::X> + Sized,
B2: rand::distributions::uniform::SampleBorrow<Self::X> + Sized,
{
Self(
<u64 as rand::distributions::uniform::SampleUniform>::Sampler::new(
low.borrow().0,
high.borrow().0,
),
)
}
fn new_inclusive<B1, B2>(low: B1, high: B2) -> Self
where
B1: rand::distributions::uniform::SampleBorrow<Self::X> + Sized,
B2: rand::distributions::uniform::SampleBorrow<Self::X> + Sized,
{
Self(
<u64 as rand::distributions::uniform::SampleUniform>::Sampler::new_inclusive(
low.borrow().0,
high.borrow().0,
),
)
}
fn sample<R: rand::prelude::Rng + ?Sized>(&self, rng: &mut R) -> Self::X {
Lsn(self.0.sample(rng))
}
}
#[cfg(test)]
mod tests {
use crate::bin_ser::BeSer;

View File

@@ -64,6 +64,18 @@ impl Client {
resp.json().await.map_err(Error::ReceiveBody)
}
pub async fn tenant_details(
&self,
tenant_id: TenantId,
) -> Result<pageserver_api::models::TenantDetails> {
let uri = format!("{}/v1/tenant/{tenant_id}", self.mgmt_api_endpoint);
self.get(uri)
.await?
.json()
.await
.map_err(Error::ReceiveBody)
}
pub async fn list_timelines(
&self,
tenant_id: TenantId,

View File

@@ -0,0 +1,26 @@
[package]
name = "pagebench"
version = "0.1.0"
edition.workspace = true
license.workspace = true
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
anyhow.workspace = true
clap.workspace = true
futures.workspace = true
hdrhistogram.workspace = true
humantime.workspace = true
humantime-serde.workspace = true
rand.workspace = true
serde.workspace = true
serde_json.workspace = true
tracing.workspace = true
tokio.workspace = true
pageserver = { path = ".." }
pageserver_client.workspace = true
pageserver_api.workspace = true
utils = { path = "../../libs/utils/" }
workspace_hack = { version = "0.1", path = "../../workspace_hack" }

View File

@@ -0,0 +1,273 @@
use anyhow::Context;
use pageserver_client::page_service::BasebackupRequest;
use utils::lsn::Lsn;
use rand::prelude::*;
use tokio::sync::Barrier;
use tokio::task::JoinSet;
use tracing::{debug, info, instrument};
use std::collections::HashMap;
use std::num::NonZeroUsize;
use std::ops::Range;
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
use std::sync::{Arc, Mutex};
use std::time::Instant;
use crate::cli;
use crate::util::tenant_timeline_id::TenantTimelineId;
use crate::util::tokio_thread_local_stats::AllThreadLocalStats;
use crate::util::{request_stats, tokio_thread_local_stats};
/// basebackup@LatestLSN
#[derive(clap::Parser)]
pub(crate) struct Args {
#[clap(long, default_value = "http://localhost:9898")]
mgmt_api_endpoint: String,
#[clap(long, default_value = "localhost:64000")]
page_service_host_port: String,
#[clap(long)]
pageserver_jwt: Option<String>,
#[clap(long, default_value = "1")]
num_clients: NonZeroUsize,
#[clap(long, default_value = "1.0")]
gzip_probability: f64,
#[clap(long)]
runtime: Option<humantime::Duration>,
#[clap(long)]
limit_to_first_n_targets: Option<usize>,
targets: Option<Vec<TenantTimelineId>>,
}
#[derive(Debug, Default)]
struct LiveStats {
completed_requests: AtomicU64,
}
impl LiveStats {
fn inc(&self) {
self.completed_requests.fetch_add(1, Ordering::Relaxed);
}
}
struct Target {
timeline: TenantTimelineId,
lsn_range: Option<Range<Lsn>>,
}
#[derive(serde::Serialize)]
struct Output {
total: request_stats::Output,
}
tokio_thread_local_stats::declare!(STATS: request_stats::Stats);
pub(crate) fn main(args: Args) -> anyhow::Result<()> {
tokio_thread_local_stats::main!(STATS, move |thread_local_stats| {
main_impl(args, thread_local_stats)
})
}
async fn main_impl(
args: Args,
all_thread_local_stats: AllThreadLocalStats<request_stats::Stats>,
) -> anyhow::Result<()> {
let args: &'static Args = Box::leak(Box::new(args));
let mgmt_api_client = Arc::new(pageserver_client::mgmt_api::Client::new(
args.mgmt_api_endpoint.clone(),
args.pageserver_jwt.as_deref(),
));
// discover targets
let timelines: Vec<TenantTimelineId> = cli::targets::discover(
&mgmt_api_client,
cli::targets::Spec {
limit_to_first_n_targets: args.limit_to_first_n_targets,
targets: args.targets.clone(),
},
)
.await?;
let mut js = JoinSet::new();
for timeline in &timelines {
js.spawn({
let timeline = *timeline;
// FIXME: this triggers initial logical size calculation
// https://github.com/neondatabase/neon/issues/6168
let info = mgmt_api_client
.timeline_info(timeline.tenant_id, timeline.timeline_id)
.await
.unwrap();
async move {
anyhow::Ok(Target {
timeline,
// TODO: support lsn_range != latest LSN
lsn_range: Some(info.last_record_lsn..(info.last_record_lsn + 1)),
})
}
});
}
let mut all_targets: Vec<Target> = Vec::new();
while let Some(res) = js.join_next().await {
all_targets.push(res.unwrap().unwrap());
}
let live_stats = Arc::new(LiveStats::default());
let num_client_tasks = timelines.len();
let num_live_stats_dump = 1;
let num_work_sender_tasks = 1;
let start_work_barrier = Arc::new(tokio::sync::Barrier::new(
num_client_tasks + num_live_stats_dump + num_work_sender_tasks,
));
let all_work_done_barrier = Arc::new(tokio::sync::Barrier::new(num_client_tasks));
tokio::spawn({
let stats = Arc::clone(&live_stats);
let start_work_barrier = Arc::clone(&start_work_barrier);
async move {
start_work_barrier.wait().await;
loop {
let start = std::time::Instant::now();
tokio::time::sleep(std::time::Duration::from_secs(1)).await;
let completed_requests = stats.completed_requests.swap(0, Ordering::Relaxed);
let elapsed = start.elapsed();
info!(
"RPS: {:.0}",
completed_requests as f64 / elapsed.as_secs_f64()
);
}
}
});
let mut work_senders = HashMap::new();
let mut tasks = Vec::new();
for tl in &timelines {
let (sender, receiver) = tokio::sync::mpsc::channel(1); // TODO: not sure what the implications of this are
work_senders.insert(tl, sender);
tasks.push(tokio::spawn(client(
args,
*tl,
Arc::clone(&start_work_barrier),
receiver,
Arc::clone(&all_work_done_barrier),
Arc::clone(&live_stats),
)));
}
let work_sender = async move {
start_work_barrier.wait().await;
loop {
let (timeline, work) = {
let mut rng = rand::thread_rng();
let target = all_targets.choose(&mut rng).unwrap();
let lsn = target.lsn_range.clone().map(|r| rng.gen_range(r));
(
target.timeline,
Work {
lsn,
gzip: rng.gen_bool(args.gzip_probability),
},
)
};
let sender = work_senders.get(&timeline).unwrap();
// TODO: what if this blocks?
sender.send(work).await.ok().unwrap();
}
};
if let Some(runtime) = args.runtime {
match tokio::time::timeout(runtime.into(), work_sender).await {
Ok(()) => unreachable!("work sender never terminates"),
Err(_timeout) => {
// this implicitly drops the work_senders, making all the clients exit
}
}
} else {
work_sender.await;
unreachable!("work sender never terminates");
}
for t in tasks {
t.await.unwrap();
}
let output = Output {
total: {
let mut agg_stats = request_stats::Stats::new();
for stats in all_thread_local_stats.lock().unwrap().iter() {
let stats = stats.lock().unwrap();
agg_stats.add(&stats);
}
agg_stats.output()
},
};
let output = serde_json::to_string_pretty(&output).unwrap();
println!("{output}");
anyhow::Ok(())
}
#[derive(Copy, Clone)]
struct Work {
lsn: Option<Lsn>,
gzip: bool,
}
#[instrument(skip_all)]
async fn client(
args: &'static Args,
timeline: TenantTimelineId,
start_work_barrier: Arc<Barrier>,
mut work: tokio::sync::mpsc::Receiver<Work>,
all_work_done_barrier: Arc<Barrier>,
live_stats: Arc<LiveStats>,
) {
start_work_barrier.wait().await;
let client = pageserver_client::page_service::Client::new(crate::util::connstring::connstring(
&args.page_service_host_port,
args.pageserver_jwt.as_deref(),
))
.await
.unwrap();
while let Some(Work { lsn, gzip }) = work.recv().await {
let start = Instant::now();
let copy_out_stream = client
.basebackup(&BasebackupRequest {
tenant_id: timeline.tenant_id,
timeline_id: timeline.timeline_id,
lsn,
gzip,
})
.await
.with_context(|| format!("start basebackup for {timeline}"))
.unwrap();
use futures::StreamExt;
let size = Arc::new(AtomicUsize::new(0));
copy_out_stream
.for_each({
|r| {
let size = Arc::clone(&size);
async move {
let size = Arc::clone(&size);
size.fetch_add(r.unwrap().len(), Ordering::Relaxed);
}
}
})
.await;
debug!("basebackup size is {} bytes", size.load(Ordering::Relaxed));
let elapsed = start.elapsed();
live_stats.inc();
STATS.with(|stats| {
stats.borrow().lock().unwrap().observe(elapsed).unwrap();
});
}
all_work_done_barrier.wait().await;
}

View File

@@ -0,0 +1 @@
pub(crate) mod targets;

View File

@@ -0,0 +1,37 @@
use std::sync::Arc;
use pageserver_client::mgmt_api;
use tracing::info;
use crate::util::{
discover_timelines::get_pageserver_tenant_timelines, tenant_timeline_id::TenantTimelineId,
};
pub(crate) struct Spec {
pub(crate) limit_to_first_n_targets: Option<usize>,
pub(crate) targets: Option<Vec<TenantTimelineId>>,
}
pub(crate) async fn discover(
api_client: &Arc<mgmt_api::Client>,
spec: Spec,
) -> anyhow::Result<Vec<TenantTimelineId>> {
let mut timelines = if let Some(targets) = spec.targets {
targets
} else {
get_pageserver_tenant_timelines(api_client).await?
};
if let Some(limit) = spec.limit_to_first_n_targets {
timelines.sort(); // for determinism
timelines.truncate(limit);
if timelines.len() < limit {
anyhow::bail!("pageserver has less than limit_to_first_n_targets={limit} tenants");
}
}
info!("timelines:\n{:?}", timelines);
info!("number of timelines:\n{:?}", timelines.len());
Ok(timelines)
}

View File

@@ -0,0 +1,341 @@
use anyhow::Context;
use futures::future::join_all;
use pageserver::pgdatadir_mapping::key_to_rel_block;
use pageserver::repository;
use pageserver_api::key::is_rel_block_key;
use pageserver_client::page_service::RelTagBlockNo;
use utils::lsn::Lsn;
use rand::prelude::*;
use tokio::sync::Barrier;
use tokio::task::JoinSet;
use tracing::{info, instrument};
use std::collections::HashMap;
use std::future::Future;
use std::num::NonZeroUsize;
use std::pin::Pin;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::{Arc, Mutex};
use std::time::{Duration, Instant};
use crate::cli;
use crate::util::tenant_timeline_id::TenantTimelineId;
use crate::util::tokio_thread_local_stats::AllThreadLocalStats;
use crate::util::{request_stats, tokio_thread_local_stats};
/// GetPage@LatestLSN, uniformly distributed across the compute-accessible keyspace.
#[derive(clap::Parser)]
pub(crate) struct Args {
#[clap(long, default_value = "http://localhost:9898")]
mgmt_api_endpoint: String,
#[clap(long, default_value = "postgres://postgres@localhost:64000")]
page_service_connstring: String,
#[clap(long)]
pageserver_jwt: Option<String>,
#[clap(long, default_value = "1")]
num_clients: NonZeroUsize,
#[clap(long)]
runtime: Option<humantime::Duration>,
#[clap(long)]
per_target_rate_limit: Option<usize>,
#[clap(long)]
limit_to_first_n_targets: Option<usize>,
targets: Option<Vec<TenantTimelineId>>,
}
#[derive(Debug, Default)]
struct LiveStats {
completed_requests: AtomicU64,
}
impl LiveStats {
fn inc(&self) {
self.completed_requests.fetch_add(1, Ordering::Relaxed);
}
}
#[derive(Clone)]
struct KeyRange {
timeline: TenantTimelineId,
timeline_lsn: Lsn,
start: i128,
end: i128,
}
impl KeyRange {
fn len(&self) -> i128 {
self.end - self.start
}
}
#[derive(serde::Serialize)]
struct Output {
total: request_stats::Output,
}
tokio_thread_local_stats::declare!(STATS: request_stats::Stats);
pub(crate) fn main(args: Args) -> anyhow::Result<()> {
tokio_thread_local_stats::main!(STATS, move |thread_local_stats| {
main_impl(args, thread_local_stats)
})
}
async fn main_impl(
args: Args,
all_thread_local_stats: AllThreadLocalStats<request_stats::Stats>,
) -> anyhow::Result<()> {
let args: &'static Args = Box::leak(Box::new(args));
let mgmt_api_client = Arc::new(pageserver_client::mgmt_api::Client::new(
args.mgmt_api_endpoint.clone(),
args.pageserver_jwt.as_deref(),
));
// discover targets
let timelines: Vec<TenantTimelineId> = cli::targets::discover(
&mgmt_api_client,
cli::targets::Spec {
limit_to_first_n_targets: args.limit_to_first_n_targets,
targets: args.targets.clone(),
},
)
.await?;
let mut js = JoinSet::new();
for timeline in &timelines {
js.spawn({
let mgmt_api_client = Arc::clone(&mgmt_api_client);
let timeline = *timeline;
async move {
let partitioning = mgmt_api_client
.keyspace(timeline.tenant_id, timeline.timeline_id)
.await?;
let lsn = partitioning.at_lsn;
let ranges = partitioning
.keys
.ranges
.iter()
.filter_map(|r| {
let start = r.start;
let end = r.end;
// filter out non-relblock keys
match (is_rel_block_key(&start), is_rel_block_key(&end)) {
(true, true) => Some(KeyRange {
timeline,
timeline_lsn: lsn,
start: start.to_i128(),
end: end.to_i128(),
}),
(true, false) | (false, true) => {
unimplemented!("split up range")
}
(false, false) => None,
}
})
.collect::<Vec<_>>();
anyhow::Ok(ranges)
}
});
}
let mut all_ranges: Vec<KeyRange> = Vec::new();
while let Some(res) = js.join_next().await {
all_ranges.extend(res.unwrap().unwrap());
}
let live_stats = Arc::new(LiveStats::default());
let num_client_tasks = timelines.len();
let num_live_stats_dump = 1;
let num_work_sender_tasks = 1;
let start_work_barrier = Arc::new(tokio::sync::Barrier::new(
num_client_tasks + num_live_stats_dump + num_work_sender_tasks,
));
let all_work_done_barrier = Arc::new(tokio::sync::Barrier::new(num_client_tasks));
tokio::spawn({
let stats = Arc::clone(&live_stats);
let start_work_barrier = Arc::clone(&start_work_barrier);
async move {
start_work_barrier.wait().await;
loop {
let start = std::time::Instant::now();
tokio::time::sleep(std::time::Duration::from_secs(1)).await;
let completed_requests = stats.completed_requests.swap(0, Ordering::Relaxed);
let elapsed = start.elapsed();
info!(
"RPS: {:.0}",
completed_requests as f64 / elapsed.as_secs_f64()
);
}
}
});
let mut work_senders = HashMap::new();
let mut tasks = Vec::new();
for tl in &timelines {
let (sender, receiver) = tokio::sync::mpsc::channel(10); // TODO: not sure what the implications of this are
work_senders.insert(tl, sender);
tasks.push(tokio::spawn(client(
args,
*tl,
Arc::clone(&start_work_barrier),
receiver,
Arc::clone(&all_work_done_barrier),
Arc::clone(&live_stats),
)));
}
let work_sender: Pin<Box<dyn Send + Future<Output = ()>>> = match args.per_target_rate_limit {
None => Box::pin(async move {
let weights = rand::distributions::weighted::WeightedIndex::new(
all_ranges.iter().map(|v| v.len()),
)
.unwrap();
start_work_barrier.wait().await;
loop {
let (range, key) = {
let mut rng = rand::thread_rng();
let r = &all_ranges[weights.sample(&mut rng)];
let key: i128 = rng.gen_range(r.start..r.end);
let key = repository::Key::from_i128(key);
if key.field6 == 0xffffffff {
// Hack around bug
continue;
}
let (rel_tag, block_no) =
key_to_rel_block(key).expect("we filter non-rel-block keys out above");
(r, RelTagBlockNo { rel_tag, block_no })
};
let sender = work_senders.get(&range.timeline).unwrap();
// TODO: what if this blocks?
sender.send((key, range.timeline_lsn)).await.ok().unwrap();
}
}),
Some(rps_limit) => Box::pin(async move {
let period = Duration::from_secs_f64(1.0 / (rps_limit as f64));
let make_timeline_task: &dyn Fn(
TenantTimelineId,
)
-> Pin<Box<dyn Send + Future<Output = ()>>> = &|timeline| {
let sender = work_senders.get(&timeline).unwrap();
let ranges: Vec<KeyRange> = all_ranges
.iter()
.filter(|r| r.timeline == timeline)
.cloned()
.collect();
let weights = rand::distributions::weighted::WeightedIndex::new(
ranges.iter().map(|v| v.len()),
)
.unwrap();
Box::pin(async move {
let mut ticker = tokio::time::interval(period);
ticker.set_missed_tick_behavior(
/* TODO review this choice */
tokio::time::MissedTickBehavior::Burst,
);
loop {
ticker.tick().await;
let (range, key) = {
let mut rng = rand::thread_rng();
let r = &ranges[weights.sample(&mut rng)];
let key: i128 = rng.gen_range(r.start..r.end);
let key = repository::Key::from_i128(key);
let (rel_tag, block_no) = key_to_rel_block(key)
.expect("we filter non-rel-block keys out above");
(r, RelTagBlockNo { rel_tag, block_no })
};
sender.send((key, range.timeline_lsn)).await.ok().unwrap();
}
})
};
let tasks: Vec<_> = work_senders
.keys()
.map(|tl| make_timeline_task(**tl))
.collect();
start_work_barrier.wait().await;
join_all(tasks).await;
}),
};
if let Some(runtime) = args.runtime {
match tokio::time::timeout(runtime.into(), work_sender).await {
Ok(()) => unreachable!("work sender never terminates"),
Err(_timeout) => {
// this implicitly drops the work_senders, making all the clients exit
}
}
} else {
work_sender.await;
unreachable!("work sender never terminates");
}
for t in tasks {
t.await.unwrap();
}
let output = Output {
total: {
let mut agg_stats = request_stats::Stats::new();
for stats in all_thread_local_stats.lock().unwrap().iter() {
let stats = stats.lock().unwrap();
agg_stats.add(&stats);
}
agg_stats.output()
},
};
let output = serde_json::to_string_pretty(&output).unwrap();
println!("{output}");
anyhow::Ok(())
}
#[instrument(skip_all)]
async fn client(
args: &'static Args,
timeline: TenantTimelineId,
start_work_barrier: Arc<Barrier>,
mut work: tokio::sync::mpsc::Receiver<(RelTagBlockNo, Lsn)>,
all_work_done_barrier: Arc<Barrier>,
live_stats: Arc<LiveStats>,
) {
start_work_barrier.wait().await;
let client = pageserver_client::page_service::Client::new(args.page_service_connstring.clone())
.await
.unwrap();
let mut client = client
.pagestream(timeline.tenant_id, timeline.timeline_id)
.await
.unwrap();
while let Some((key, lsn)) = work.recv().await {
let start = Instant::now();
client
.getpage(key, lsn)
.await
.with_context(|| format!("getpage for {timeline}"))
.unwrap();
let elapsed = start.elapsed();
live_stats.inc();
STATS.with(|stats| {
stats.borrow().lock().unwrap().observe(elapsed).unwrap();
});
}
all_work_done_barrier.wait().await;
}

View File

@@ -0,0 +1,34 @@
use clap::Parser;
use utils::logging;
pub(crate) mod cli;
pub(crate) mod util;
mod basebackup;
mod getpage_latest_lsn;
mod trigger_initial_size_calculation;
/// Component-level performance test for pageserver.
#[derive(clap::Parser)]
enum Args {
Basebackup(basebackup::Args),
GetPageLatestLsn(getpage_latest_lsn::Args),
TriggerInitialSizeCalculation(trigger_initial_size_calculation::Args),
}
fn main() {
logging::init(
logging::LogFormat::Plain,
logging::TracingErrorLayerEnablement::Disabled,
logging::Output::Stderr,
)
.unwrap();
let args = Args::parse();
match args {
Args::Basebackup(args) => basebackup::main(args),
Args::GetPageLatestLsn(args) => getpage_latest_lsn::main(args),
Args::TriggerInitialSizeCalculation(args) => trigger_initial_size_calculation::main(args),
}
.unwrap()
}

View File

@@ -0,0 +1,86 @@
use std::sync::Arc;
use humantime::Duration;
use tokio::task::JoinSet;
use crate::{cli, util::tenant_timeline_id::TenantTimelineId};
#[derive(clap::Parser)]
pub(crate) struct Args {
#[clap(long, default_value = "http://localhost:9898")]
mgmt_api_endpoint: String,
#[clap(long, default_value = "localhost:64000")]
page_service_host_port: String,
#[clap(long)]
pageserver_jwt: Option<String>,
#[clap(
long,
help = "if specified, poll mgmt api to check whether init logical size calculation has completed"
)]
poll_for_completion: Option<Duration>,
#[clap(long)]
limit_to_first_n_targets: Option<usize>,
targets: Option<Vec<TenantTimelineId>>,
}
pub(crate) fn main(args: Args) -> anyhow::Result<()> {
let rt = tokio::runtime::Builder::new_multi_thread()
.enable_all()
.build()
.unwrap();
let main_task = rt.spawn(main_impl(args));
rt.block_on(main_task).unwrap()
}
async fn main_impl(args: Args) -> anyhow::Result<()> {
let args: &'static Args = Box::leak(Box::new(args));
let mgmt_api_client = Arc::new(pageserver_client::mgmt_api::Client::new(
args.mgmt_api_endpoint.clone(),
args.pageserver_jwt.as_deref(),
));
// discover targets
let timelines: Vec<TenantTimelineId> = cli::targets::discover(
&mgmt_api_client,
cli::targets::Spec {
limit_to_first_n_targets: args.limit_to_first_n_targets,
targets: args.targets.clone(),
},
)
.await?;
// kick it off
let mut js = JoinSet::new();
for tl in timelines {
let mgmt_api_client = Arc::clone(&mgmt_api_client);
js.spawn(async move {
// TODO: API to explicitly trigger initial logical size computation.
// Should probably also avoid making it a side effect of timeline details to trigger initial logical size calculation.
// => https://github.com/neondatabase/neon/issues/6168
let info = mgmt_api_client
.timeline_info(tl.tenant_id, tl.timeline_id)
.await
.unwrap();
if let Some(period) = args.poll_for_completion {
let mut ticker = tokio::time::interval(period.into());
ticker.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay);
let mut info = info;
while !info.current_logical_size_is_accurate {
ticker.tick().await;
info = mgmt_api_client
.timeline_info(tl.tenant_id, tl.timeline_id)
.await
.unwrap();
}
}
});
}
while let Some(res) = js.join_next().await {
let _: () = res.unwrap();
}
Ok(())
}

View File

@@ -0,0 +1,6 @@
pub(crate) mod connstring;
pub(crate) mod discover_timelines;
pub(crate) mod request_stats;
pub(crate) mod tenant_timeline_id;
#[macro_use]
pub(crate) mod tokio_thread_local_stats;

View File

@@ -0,0 +1,8 @@
pub(crate) fn connstring(host_port: &str, jwt: Option<&str>) -> String {
let colon_and_jwt = if let Some(jwt) = jwt {
format!(":{jwt}") // TODO: urlescape
} else {
String::new()
};
format!("postgres://postgres{colon_and_jwt}@{host_port}")
}

View File

@@ -0,0 +1,45 @@
use std::sync::Arc;
use pageserver_client::mgmt_api;
use tokio::task::JoinSet;
use utils::id::TenantId;
use super::tenant_timeline_id::TenantTimelineId;
pub(crate) async fn get_pageserver_tenant_timelines(
api_client: &Arc<mgmt_api::Client>,
) -> anyhow::Result<Vec<TenantTimelineId>> {
let mut timelines: Vec<TenantTimelineId> = Vec::new();
let mut tenants: Vec<TenantId> = Vec::new();
for ti in api_client.list_tenants().await? {
if !ti.id.is_unsharded() {
anyhow::bail!(
"only unsharded tenants are supported at this time: {}",
ti.id
);
}
tenants.push(ti.id.tenant_id)
}
let mut js = JoinSet::new();
for tenant_id in tenants {
js.spawn({
let mgmt_api_client = Arc::clone(api_client);
async move {
(
tenant_id,
mgmt_api_client.tenant_details(tenant_id).await.unwrap(),
)
}
});
}
while let Some(res) = js.join_next().await {
let (tenant_id, details) = res.unwrap();
for timeline_id in details.timelines {
timelines.push(TenantTimelineId {
tenant_id,
timeline_id,
});
}
}
Ok(timelines)
}

View File

@@ -0,0 +1,88 @@
use std::time::Duration;
use anyhow::Context;
pub(crate) struct Stats {
latency_histo: hdrhistogram::Histogram<u64>,
}
impl Stats {
pub(crate) fn new() -> Self {
Self {
// Initialize with fixed bounds so that we panic at runtime instead of resizing the histogram,
// which would skew the benchmark results.
latency_histo: hdrhistogram::Histogram::new_with_bounds(1, 1_000_000_000, 3).unwrap(),
}
}
pub(crate) fn observe(&mut self, latency: Duration) -> anyhow::Result<()> {
let micros: u64 = latency
.as_micros()
.try_into()
.context("latency greater than u64")?;
self.latency_histo
.record(micros)
.context("add to histogram")?;
Ok(())
}
pub(crate) fn output(&self) -> Output {
let latency_percentiles = std::array::from_fn(|idx| {
let micros = self
.latency_histo
.value_at_percentile(LATENCY_PERCENTILES[idx]);
Duration::from_micros(micros)
});
Output {
request_count: self.latency_histo.len(),
latency_mean: Duration::from_micros(self.latency_histo.mean() as u64),
latency_percentiles: LatencyPercentiles {
latency_percentiles,
},
}
}
pub(crate) fn add(&mut self, other: &Self) {
let Self {
ref mut latency_histo,
} = self;
latency_histo.add(&other.latency_histo).unwrap();
}
}
impl Default for Stats {
fn default() -> Self {
Self::new()
}
}
const LATENCY_PERCENTILES: [f64; 4] = [95.0, 99.00, 99.90, 99.99];
struct LatencyPercentiles {
latency_percentiles: [Duration; 4],
}
impl serde::Serialize for LatencyPercentiles {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
use serde::ser::SerializeMap;
let mut ser = serializer.serialize_map(Some(LATENCY_PERCENTILES.len()))?;
for p in LATENCY_PERCENTILES {
ser.serialize_entry(
&format!("p{p}"),
&format!(
"{}",
&humantime::format_duration(self.latency_percentiles[0])
),
)?;
}
ser.end()
}
}
#[derive(serde::Serialize)]
pub(crate) struct Output {
request_count: u64,
#[serde(with = "humantime_serde")]
latency_mean: Duration,
latency_percentiles: LatencyPercentiles,
}

View File

@@ -0,0 +1,34 @@
use std::str::FromStr;
use anyhow::Context;
use utils::id::{TenantId, TimelineId};
#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy, PartialOrd, Ord)]
pub(crate) struct TenantTimelineId {
pub(crate) tenant_id: TenantId,
pub(crate) timeline_id: TimelineId,
}
impl FromStr for TenantTimelineId {
type Err = anyhow::Error;
fn from_str(s: &str) -> Result<Self, Self::Err> {
let (tenant_id, timeline_id) = s
.split_once('/')
.context("tenant and timeline id must be separated by `/`")?;
let tenant_id = TenantId::from_str(tenant_id)
.with_context(|| format!("invalid tenant id: {tenant_id:?}"))?;
let timeline_id = TimelineId::from_str(timeline_id)
.with_context(|| format!("invalid timeline id: {timeline_id:?}"))?;
Ok(Self {
tenant_id,
timeline_id,
})
}
}
impl std::fmt::Display for TenantTimelineId {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{}/{}", self.tenant_id, self.timeline_id)
}
}

View File

@@ -0,0 +1,45 @@
pub(crate) type ThreadLocalStats<T> = Arc<Mutex<T>>;
pub(crate) type AllThreadLocalStats<T> = Arc<Mutex<Vec<ThreadLocalStats<T>>>>;
macro_rules! declare {
($THREAD_LOCAL_NAME:ident: $T:ty) => {
thread_local! {
pub static $THREAD_LOCAL_NAME: std::cell::RefCell<crate::util::tokio_thread_local_stats::ThreadLocalStats<$T>> = std::cell::RefCell::new(
std::sync::Arc::new(std::sync::Mutex::new(Default::default()))
);
}
};
}
use std::sync::{Arc, Mutex};
pub(crate) use declare;
macro_rules! main {
($THREAD_LOCAL_NAME:ident, $main_impl:expr) => {{
let main_impl = $main_impl;
let all = Arc::new(Mutex::new(Vec::new()));
let rt = tokio::runtime::Builder::new_multi_thread()
.on_thread_start({
let all = Arc::clone(&all);
move || {
// pre-initialize the thread local stats by accessesing them
// (some stats like requests_stats::Stats are quite costly to initialize,
// we don't want to pay that cost during the measurement period)
$THREAD_LOCAL_NAME.with(|stats| {
let stats: Arc<_> = Arc::clone(&*stats.borrow());
all.lock().unwrap().push(stats);
});
}
})
.enable_all()
.build()
.unwrap();
let main_task = rt.spawn(main_impl(all));
rt.block_on(main_task).unwrap()
}};
}
pub(crate) use main;

View File

@@ -14,6 +14,7 @@ use hyper::header;
use hyper::StatusCode;
use hyper::{Body, Request, Response, Uri};
use metrics::launch_timestamp::LaunchTimestamp;
use pageserver_api::models::TenantDetails;
use pageserver_api::models::{
DownloadRemoteLayersTaskSpawnRequest, LocationConfigMode, TenantAttachRequest,
TenantLoadRequest, TenantLocationConfigRequest,
@@ -872,11 +873,14 @@ async fn tenant_status(
}
let state = tenant.current_state();
Result::<_, ApiError>::Ok(TenantInfo {
id: tenant_shard_id,
state: state.clone(),
current_physical_size: Some(current_physical_size),
attachment_status: state.attachment_status(),
Result::<_, ApiError>::Ok(TenantDetails {
tenant_info: TenantInfo {
id: tenant_shard_id,
state: state.clone(),
current_physical_size: Some(current_physical_size),
attachment_status: state.attachment_status(),
},
timelines: tenant.list_timeline_ids(),
})
}
.instrument(info_span!("tenant_status_handler",

View File

@@ -343,15 +343,6 @@ pub(crate) mod page_cache_eviction_metrics {
}
}
pub(crate) static PAGE_CACHE_ACQUIRE_PINNED_SLOT_TIME: Lazy<Histogram> = Lazy::new(|| {
register_histogram!(
"pageserver_page_cache_acquire_pinned_slot_seconds",
"Time spent acquiring a pinned slot in the page cache",
CRITICAL_OP_BUCKETS.into(),
)
.expect("failed to define a metric")
});
static PAGE_CACHE_ERRORS: Lazy<IntCounterVec> = Lazy::new(|| {
register_int_counter_vec!(
"page_cache_errors_total",

View File

@@ -550,7 +550,6 @@ impl PageCache {
// not require changes.
async fn try_get_pinned_slot_permit(&self) -> anyhow::Result<PinnedSlotsPermit> {
let timer = crate::metrics::PAGE_CACHE_ACQUIRE_PINNED_SLOT_TIME.start_timer();
match tokio::time::timeout(
// Choose small timeout, neon_smgr does its own retries.
// https://neondb.slack.com/archives/C04DGM6SMTM/p1694786876476869
@@ -563,7 +562,6 @@ impl PageCache {
res.expect("this semaphore is never closed"),
)),
Err(_timeout) => {
timer.stop_and_discard();
crate::metrics::page_cache_errors_inc(
crate::metrics::PageCacheErrorKind::AcquirePinnedSlotTimeout,
);

View File

@@ -173,14 +173,25 @@ impl Timeline {
));
}
let nblocks = self.get_rel_size(tag, lsn, latest, ctx).await?;
if blknum >= nblocks {
debug!(
"read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page",
tag, blknum, lsn, nblocks
);
return Ok(ZERO_PAGE.clone());
}
// let nblocks = self.get_rel_size(tag, lsn, latest, ctx).await?;
// if blknum >= nblocks {
// tracing::info!(
// "read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page",
// tag,
// blknum,
// lsn,
// nblocks
// );
// return Ok(ZERO_PAGE.clone());
// } else {
// tracing::info!(
// "read within bounds at {} blk {} at {}, size is {}",
// tag,
// blknum,
// lsn,
// nblocks
// );
// }
let key = rel_block_to_key(tag, blknum);
self.get(key, lsn, ctx).await
@@ -235,6 +246,7 @@ impl Timeline {
}
let key = rel_size_to_key(tag);
tracing::info!("rel size cache miss {tag} {lsn} {latest}");
let mut buf = self.get(key, lsn, ctx).await?;
let nblocks = buf.get_u32_le();
@@ -1776,6 +1788,7 @@ pub fn is_inherited_key(key: Key) -> bool {
key != AUX_FILES_KEY
}
/// Guaranteed to return `Ok()` if [[is_rel_block_key]] returns `true` for `key`.
pub fn key_to_rel_block(key: Key) -> anyhow::Result<(RelTag, BlockNumber)> {
Ok(match key.field1 {
0x00 => (
@@ -1790,7 +1803,6 @@ pub fn key_to_rel_block(key: Key) -> anyhow::Result<(RelTag, BlockNumber)> {
_ => anyhow::bail!("unexpected value kind 0x{:02x}", key.field1),
})
}
pub fn is_rel_fsm_block_key(key: Key) -> bool {
key.field1 == 0x00 && key.field4 != 0 && key.field5 == FSM_FORKNUM && key.field6 != 0xffffffff
}

View File

@@ -1552,6 +1552,10 @@ impl Tenant {
.collect()
}
pub fn list_timeline_ids(&self) -> Vec<TimelineId> {
self.timelines.lock().unwrap().keys().cloned().collect()
}
/// This is used to create the initial 'main' timeline during bootstrapping,
/// or when importing a new base backup. The caller is expected to load an
/// initial image of the datadir to the new timeline after this.

View File

@@ -22,6 +22,8 @@ pub trait BlockReader {
/// A cursor caches the last accessed page, allowing for faster
/// access if the same block is accessed repeatedly.
fn block_cursor(&self) -> BlockCursor<'_>;
fn block_cursor_direct(&self) -> BlockCursor<'_>;
}
impl<B> BlockReader for &B
@@ -31,12 +33,17 @@ where
fn block_cursor(&self) -> BlockCursor<'_> {
(*self).block_cursor()
}
fn block_cursor_direct(&self) -> BlockCursor<'_> {
(*self).block_cursor()
}
}
/// Reference to an in-memory copy of an immutable on-disk block.
pub enum BlockLease<'a> {
PageReadGuard(PageReadGuard<'static>),
EphemeralFileMutableTail(&'a [u8; PAGE_SZ]),
Direct(bytes::Bytes),
#[cfg(test)]
Arc(std::sync::Arc<[u8; PAGE_SZ]>),
}
@@ -61,6 +68,7 @@ impl<'a> Deref for BlockLease<'a> {
match self {
BlockLease::PageReadGuard(v) => v.deref(),
BlockLease::EphemeralFileMutableTail(v) => v,
BlockLease::Direct(b) => <&[u8; PAGE_SZ]>::try_from(b as &[u8]).unwrap(),
#[cfg(test)]
BlockLease::Arc(v) => v.deref(),
}
@@ -99,6 +107,24 @@ impl<'a> BlockReaderRef<'a> {
VirtualFile(r) => r.read_blk(blknum).await,
}
}
#[inline(always)]
async fn read_blk_direct(
&self,
blknum: u32,
ctx: &RequestContext,
) -> Result<BlockLease, std::io::Error> {
use BlockReaderRef::*;
match self {
FileBlockReader(r) => r.read_blk_direct(blknum, ctx).await,
EphemeralFile(r) => r.read_blk(blknum, ctx).await,
Adapter(r) => r.read_blk(blknum, ctx).await,
#[cfg(test)]
TestDisk(r) => r.read_blk(blknum),
#[cfg(test)]
VirtualFile(r) => r.read_blk(blknum).await,
}
}
}
///
@@ -121,17 +147,28 @@ impl<'a> BlockReaderRef<'a> {
/// ```
///
pub struct BlockCursor<'a> {
direct: bool,
reader: BlockReaderRef<'a>,
}
impl<'a> BlockCursor<'a> {
pub(crate) fn new(reader: BlockReaderRef<'a>) -> Self {
BlockCursor { reader }
BlockCursor {
reader,
direct: false,
}
}
pub(crate) fn new_direct(reader: BlockReaderRef<'a>) -> Self {
BlockCursor {
reader,
direct: true,
}
}
// Needed by cli
pub fn new_fileblockreader(reader: &'a FileBlockReader) -> Self {
BlockCursor {
reader: BlockReaderRef::FileBlockReader(reader),
direct: false,
}
}
@@ -146,7 +183,11 @@ impl<'a> BlockCursor<'a> {
blknum: u32,
ctx: &RequestContext,
) -> Result<BlockLease, std::io::Error> {
self.reader.read_blk(blknum, ctx).await
if self.direct {
self.reader.read_blk_direct(blknum, ctx).await
} else {
self.reader.read_blk(blknum, ctx).await
}
}
}
@@ -203,12 +244,27 @@ impl FileBlockReader {
}
}
}
pub async fn read_blk_direct(
&self,
blknum: u32,
_ctx: &RequestContext,
) -> Result<BlockLease, std::io::Error> {
let mut buf = bytes::BytesMut::zeroed(PAGE_SZ);
let buffer = <&mut [u8; PAGE_SZ]>::try_from(&mut buf as &mut [u8]).unwrap();
self.fill_buffer(buffer, blknum).await?;
Ok(BlockLease::Direct(buf.into()))
}
}
impl BlockReader for FileBlockReader {
fn block_cursor(&self) -> BlockCursor<'_> {
BlockCursor::new(BlockReaderRef::FileBlockReader(self))
}
fn block_cursor_direct(&self) -> BlockCursor<'_> {
BlockCursor::new_direct(BlockReaderRef::FileBlockReader(self))
}
}
///

View File

@@ -266,6 +266,10 @@ impl BlockReader for EphemeralFile {
fn block_cursor(&self) -> super::block_io::BlockCursor<'_> {
BlockCursor::new(super::block_io::BlockReaderRef::EphemeralFile(self))
}
fn block_cursor_direct(&self) -> super::block_io::BlockCursor<'_> {
BlockCursor::new(super::block_io::BlockReaderRef::EphemeralFile(self))
}
}
#[cfg(test)]

View File

@@ -770,7 +770,7 @@ impl DeltaLayerInner {
.build();
// Ok, 'offsets' now contains the offsets of all the entries we need to read
let cursor = file.block_cursor();
let cursor = file.block_cursor_direct();
let mut buf = Vec::new();
for (entry_lsn, pos) in offsets {
cursor

View File

@@ -427,7 +427,7 @@ impl ImageLayerInner {
.await?
{
let blob = file
.block_cursor()
.block_cursor_direct()
.read_blob(
offset,
&RequestContextBuilder::extend(ctx)

View File

@@ -14,6 +14,7 @@ use enumset::EnumSet;
use fail::fail_point;
use itertools::Itertools;
use pageserver_api::{
key::is_rel_block_key,
models::{
DownloadRemoteLayersTaskInfo, DownloadRemoteLayersTaskSpawnRequest, LayerMapInfo,
TimelineState,
@@ -508,21 +509,25 @@ impl Timeline {
// The cached image can be returned directly if there is no WAL between the cached image
// and requested LSN. The cached image can also be used to reduce the amount of WAL needed
// for redo.
let cached_page_img = match self.lookup_cached_page(&key, lsn, ctx).await {
Some((cached_lsn, cached_img)) => {
match cached_lsn.cmp(&lsn) {
Ordering::Less => {} // there might be WAL between cached_lsn and lsn, we need to check
Ordering::Equal => {
MATERIALIZED_PAGE_CACHE_HIT_DIRECT.inc();
return Ok(cached_img); // exact LSN match, return the image
}
Ordering::Greater => {
unreachable!("the returned lsn should never be after the requested lsn")
let cached_page_img = if is_rel_block_key(&key) && key.field6 != 0xffffffff {
None
} else {
match self.lookup_cached_page(&key, lsn, ctx).await {
Some((cached_lsn, cached_img)) => {
match cached_lsn.cmp(&lsn) {
Ordering::Less => {} // there might be WAL between cached_lsn and lsn, we need to check
Ordering::Equal => {
MATERIALIZED_PAGE_CACHE_HIT_DIRECT.inc();
return Ok(cached_img); // exact LSN match, return the image
}
Ordering::Greater => {
unreachable!("the returned lsn should never be after the requested lsn")
}
}
Some((cached_lsn, cached_img))
}
Some((cached_lsn, cached_img))
None => None,
}
None => None,
};
let mut reconstruct_state = ValueReconstructState {
@@ -4189,7 +4194,9 @@ impl Timeline {
Err(e) => return Err(PageReconstructError::from(e)),
};
if img.len() == page_cache::PAGE_SZ {
if img.len() == page_cache::PAGE_SZ
&& !(is_rel_block_key(&key) && key.field6 != 0xffffffff)
{
let cache = page_cache::get();
if let Err(e) = cache
.memorize_materialized_page(