pageserver: support keys at different LSNs in one get page batch (#11494)

## Problem

Get page batching stops when we encounter requests at different LSNs.
We are leaving batching factor on the table.

## Summary of changes

The goal is to support keys with different LSNs in a single batch and
still serve them with a single vectored get.
Important restriction: the same key at different LSNs is not supported
in one batch. Returning different key
versions is a much more intrusive change.

Firstly, the read path is changed to support "scattered" queries. This
is a conceptually simple step from
https://github.com/neondatabase/neon/pull/11463. Instead of initializing
the fringe for one keyspace,
we do it for multiple at different LSNs and let the logic already
present into the fringe handle selection.

Secondly, page service code is updated to support batching at different
LSNs. Eeach request parsed from the wire determines its effective
request LSN and keeps it in mem for the batcher toinspect. The batcher
allows keys at
different LSNs in one batch as long one key is not requested at
different LSNs.

I'd suggest doing the first pass commit by commit to get a feel for the
changes.

## Results

I used the batching test from [Christian's
PR](https://github.com/neondatabase/neon/pull/11391) which increases the
change of batch breaks. Looking at the logs I think the new code is at
the max batching factor for the workload (we
only break batches due to them being oversized or because the executor
is idle).

```
Main:
Reasons for stopping batching: {'LSN changed': 22843, 'of batch size': 33417}
test_throughput[release-pg16-50-pipelining_config0-30-100-128-batchable {'max_batch_size': 32, 'execution': 'concurrent-futures', 'mode': 'pipelined'}].perfmetric.batching_factor: 14.6662

My branch:
Reasons for stopping batching: {'of batch size': 37024}
test_throughput[release-pg16-50-pipelining_config0-30-100-128-batchable {'max_batch_size': 32, 'execution': 'concurrent-futures', 'mode': 'pipelined'}].perfmetric.batching_factor: 19.8333
```

Related: https://github.com/neondatabase/neon/issues/10765
This commit is contained in:
Vlad Lazar
2025-04-14 10:05:29 +01:00
committed by GitHub
parent 8936a7abd8
commit a338984dc7
12 changed files with 591 additions and 265 deletions

View File

@@ -207,6 +207,10 @@ pub struct PageServicePipeliningConfigPipelined {
/// Causes runtime errors if larger than max get_vectored batch size.
pub max_batch_size: NonZeroUsize,
pub execution: PageServiceProtocolPipelinedExecutionStrategy,
// The default below is such that new versions of the software can start
// with the old configuration.
#[serde(default)]
pub batching: PageServiceProtocolPipelinedBatchingStrategy,
}
#[derive(Debug, Clone, Copy, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
@@ -216,6 +220,19 @@ pub enum PageServiceProtocolPipelinedExecutionStrategy {
Tasks,
}
#[derive(Default, Debug, Clone, Copy, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
#[serde(rename_all = "kebab-case")]
pub enum PageServiceProtocolPipelinedBatchingStrategy {
/// All get page requests in a batch will be at the same LSN
#[default]
UniformLsn,
/// Get page requests in a batch may be at different LSN
///
/// One key cannot be present more than once at different LSNs in
/// the same batch.
ScatteredLsn,
}
#[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
#[serde(tag = "mode", rename_all = "kebab-case")]
pub enum GetVectoredConcurrentIo {
@@ -615,9 +632,12 @@ impl Default for ConfigToml {
page_service_pipelining: if !cfg!(test) {
PageServicePipeliningConfig::Serial
} else {
// Do not turn this into the default until scattered reads have been
// validated and rolled-out fully.
PageServicePipeliningConfig::Pipelined(PageServicePipeliningConfigPipelined {
max_batch_size: NonZeroUsize::new(32).unwrap(),
execution: PageServiceProtocolPipelinedExecutionStrategy::ConcurrentFutures,
batching: PageServiceProtocolPipelinedBatchingStrategy::ScatteredLsn,
})
},
get_vectored_concurrent_io: if !cfg!(test) {

View File

@@ -34,7 +34,7 @@ use utils::lsn::Lsn;
use crate::context::RequestContext;
use crate::pgdatadir_mapping::Version;
use crate::tenant::storage_layer::IoConcurrency;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::timeline::{GetVectoredError, VersionedKeySpaceQuery};
use crate::tenant::{PageReconstructError, Timeline};
#[derive(Debug, thiserror::Error)]
@@ -353,9 +353,10 @@ where
let mut slru_builder = SlruSegmentsBuilder::new(&mut self.ar);
for part in slru_partitions.parts {
let query = VersionedKeySpaceQuery::uniform(part, self.lsn);
let blocks = self
.timeline
.get_vectored(part, self.lsn, self.io_concurrency.clone(), self.ctx)
.get_vectored(query, self.io_concurrency.clone(), self.ctx)
.await?;
for (key, block) in blocks {

View File

@@ -17,7 +17,7 @@ use metrics::{
use once_cell::sync::Lazy;
use pageserver_api::config::{
PageServicePipeliningConfig, PageServicePipeliningConfigPipelined,
PageServiceProtocolPipelinedExecutionStrategy,
PageServiceProtocolPipelinedBatchingStrategy, PageServiceProtocolPipelinedExecutionStrategy,
};
use pageserver_api::models::InMemoryLayerInfo;
use pageserver_api::shard::TenantShardId;
@@ -1863,7 +1863,7 @@ pub(crate) static PAGE_SERVICE_CONFIG_MAX_BATCH_SIZE: Lazy<IntGaugeVec> = Lazy::
"pageserver_page_service_config_max_batch_size",
"Configured maximum batch size for the server-side batching functionality of page_service. \
Labels expose more of the configuration parameters.",
&["mode", "execution"]
&["mode", "execution", "batching"]
)
.expect("failed to define a metric")
});
@@ -1871,10 +1871,11 @@ pub(crate) static PAGE_SERVICE_CONFIG_MAX_BATCH_SIZE: Lazy<IntGaugeVec> = Lazy::
fn set_page_service_config_max_batch_size(conf: &PageServicePipeliningConfig) {
PAGE_SERVICE_CONFIG_MAX_BATCH_SIZE.reset();
let (label_values, value) = match conf {
PageServicePipeliningConfig::Serial => (["serial", "-"], 1),
PageServicePipeliningConfig::Serial => (["serial", "-", "-"], 1),
PageServicePipeliningConfig::Pipelined(PageServicePipeliningConfigPipelined {
max_batch_size,
execution,
batching,
}) => {
let mode = "pipelined";
let execution = match execution {
@@ -1883,7 +1884,12 @@ fn set_page_service_config_max_batch_size(conf: &PageServicePipeliningConfig) {
}
PageServiceProtocolPipelinedExecutionStrategy::Tasks => "tasks",
};
([mode, execution], max_batch_size.get())
let batching = match batching {
PageServiceProtocolPipelinedBatchingStrategy::UniformLsn => "uniform-lsn",
PageServiceProtocolPipelinedBatchingStrategy::ScatteredLsn => "scattered-lsn",
};
([mode, execution, batching], max_batch_size.get())
}
};
PAGE_SERVICE_CONFIG_MAX_BATCH_SIZE

View File

@@ -18,7 +18,7 @@ use itertools::Itertools;
use once_cell::sync::OnceCell;
use pageserver_api::config::{
PageServicePipeliningConfig, PageServicePipeliningConfigPipelined,
PageServiceProtocolPipelinedExecutionStrategy,
PageServiceProtocolPipelinedBatchingStrategy, PageServiceProtocolPipelinedExecutionStrategy,
};
use pageserver_api::key::rel_block_to_key;
use pageserver_api::models::{
@@ -641,6 +641,7 @@ impl std::fmt::Display for BatchedPageStreamError {
struct BatchedGetPageRequest {
req: PagestreamGetPageRequest,
timer: SmgrOpTimer,
effective_request_lsn: Lsn,
ctx: RequestContext,
}
@@ -670,7 +671,6 @@ enum BatchedFeMessage {
GetPage {
span: Span,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
effective_request_lsn: Lsn,
pages: smallvec::SmallVec<[BatchedGetPageRequest; 1]>,
},
DbSize {
@@ -1025,34 +1025,28 @@ impl PageServerHandler {
.await?;
// We're holding the Handle
// TODO: if we actually need to wait for lsn here, it delays the entire batch which doesn't need to wait
let res = Self::wait_or_get_last_lsn(
let effective_request_lsn = match Self::effective_request_lsn(
&shard,
shard.get_last_record_lsn(),
req.hdr.request_lsn,
req.hdr.not_modified_since,
&shard.get_applied_gc_cutoff_lsn(),
&ctx,
)
.maybe_perf_instrument(&ctx, |current_perf_span| {
info_span!(
target: PERF_TRACE_TARGET,
parent: current_perf_span,
"WAIT_LSN",
)
})
.await;
let effective_request_lsn = match res {
) {
Ok(lsn) => lsn,
Err(e) => {
return respond_error!(span, e);
}
};
BatchedFeMessage::GetPage {
span,
shard: shard.downgrade(),
effective_request_lsn,
pages: smallvec::smallvec![BatchedGetPageRequest { req, timer, ctx }],
pages: smallvec::smallvec![BatchedGetPageRequest {
req,
timer,
effective_request_lsn,
ctx,
}],
}
}
#[cfg(feature = "testing")]
@@ -1078,6 +1072,7 @@ impl PageServerHandler {
#[instrument(skip_all, level = tracing::Level::TRACE)]
#[allow(clippy::boxed_local)]
fn pagestream_do_batch(
batching_strategy: PageServiceProtocolPipelinedBatchingStrategy,
max_batch_size: NonZeroUsize,
batch: &mut Result<BatchedFeMessage, QueryError>,
this_msg: Result<BatchedFeMessage, QueryError>,
@@ -1096,33 +1091,61 @@ impl PageServerHandler {
span: _,
shard: accum_shard,
pages: accum_pages,
effective_request_lsn: accum_lsn,
}),
BatchedFeMessage::GetPage {
span: _,
shard: this_shard,
pages: this_pages,
effective_request_lsn: this_lsn,
},
) if (|| {
assert_eq!(this_pages.len(), 1);
if accum_pages.len() >= max_batch_size.get() {
trace!(%accum_lsn, %this_lsn, %max_batch_size, "stopping batching because of batch size");
trace!(%max_batch_size, "stopping batching because of batch size");
assert_eq!(accum_pages.len(), max_batch_size.get());
return false;
}
if !accum_shard.is_same_handle_as(&this_shard) {
trace!(%accum_lsn, %this_lsn, "stopping batching because timeline object mismatch");
trace!("stopping batching because timeline object mismatch");
// TODO: we _could_ batch & execute each shard seperately (and in parallel).
// But the current logic for keeping responses in order does not support that.
return false;
}
// the vectored get currently only supports a single LSN, so, bounce as soon
// as the effective request_lsn changes
if *accum_lsn != this_lsn {
trace!(%accum_lsn, %this_lsn, "stopping batching because LSN changed");
return false;
match batching_strategy {
PageServiceProtocolPipelinedBatchingStrategy::UniformLsn => {
if let Some(last_in_batch) = accum_pages.last() {
if last_in_batch.effective_request_lsn
!= this_pages[0].effective_request_lsn
{
return false;
}
}
}
PageServiceProtocolPipelinedBatchingStrategy::ScatteredLsn => {
// The read path doesn't curently support serving the same page at different LSNs.
// While technically possible, it's uncertain if the complexity is worth it.
// Break the batch if such a case is encountered.
//
// TODO(vlad): Include a metric for batch breaks with a reason label.
let same_page_different_lsn = accum_pages.iter().any(|batched| {
batched.req.rel == this_pages[0].req.rel
&& batched.req.blkno == this_pages[0].req.blkno
&& batched.effective_request_lsn
!= this_pages[0].effective_request_lsn
});
if same_page_different_lsn {
trace!(
rel=%this_pages[0].req.rel,
blkno=%this_pages[0].req.blkno,
lsn=%this_pages[0].effective_request_lsn,
"stopping batching because same page was requested at different LSNs"
);
return false;
}
}
}
true
})() =>
{
@@ -1390,12 +1413,7 @@ impl PageServerHandler {
span,
)
}
BatchedFeMessage::GetPage {
span,
shard,
effective_request_lsn,
pages,
} => {
BatchedFeMessage::GetPage { span, shard, pages } => {
fail::fail_point!("ps::handle-pagerequest-message::getpage");
let (shard, ctx) = upgrade_handle_and_set_context!(shard);
(
@@ -1405,7 +1423,6 @@ impl PageServerHandler {
let res = self
.handle_get_page_at_lsn_request_batched(
&shard,
effective_request_lsn,
pages,
io_concurrency,
&ctx,
@@ -1724,6 +1741,7 @@ impl PageServerHandler {
let PageServicePipeliningConfigPipelined {
max_batch_size,
execution,
batching: batching_strategy,
} = pipelining_config;
// Macro to _define_ a pipeline stage.
@@ -1775,7 +1793,7 @@ impl PageServerHandler {
exit |= read_res.is_err();
let could_send = batch_tx
.send(read_res, |batch, res| {
Self::pagestream_do_batch(max_batch_size, batch, res)
Self::pagestream_do_batch(batching_strategy, max_batch_size, batch, res)
})
.await;
exit |= could_send.is_err();
@@ -1871,7 +1889,39 @@ impl PageServerHandler {
ctx: &RequestContext,
) -> Result<Lsn, PageStreamError> {
let last_record_lsn = timeline.get_last_record_lsn();
let effective_request_lsn = Self::effective_request_lsn(
timeline,
last_record_lsn,
request_lsn,
not_modified_since,
latest_gc_cutoff_lsn,
)?;
if effective_request_lsn > last_record_lsn {
timeline
.wait_lsn(
not_modified_since,
crate::tenant::timeline::WaitLsnWaiter::PageService,
timeline::WaitLsnTimeout::Default,
ctx,
)
.await?;
// Since we waited for 'effective_request_lsn' to arrive, that is now the last
// record LSN. (Or close enough for our purposes; the last-record LSN can
// advance immediately after we return anyway)
}
Ok(effective_request_lsn)
}
fn effective_request_lsn(
timeline: &Timeline,
last_record_lsn: Lsn,
request_lsn: Lsn,
not_modified_since: Lsn,
latest_gc_cutoff_lsn: &RcuReadGuard<Lsn>,
) -> Result<Lsn, PageStreamError> {
// Sanity check the request
if request_lsn < not_modified_since {
return Err(PageStreamError::BadRequest(
@@ -1906,19 +1956,7 @@ impl PageServerHandler {
}
}
// Wait for WAL up to 'not_modified_since' to arrive, if necessary
if not_modified_since > last_record_lsn {
timeline
.wait_lsn(
not_modified_since,
crate::tenant::timeline::WaitLsnWaiter::PageService,
timeline::WaitLsnTimeout::Default,
ctx,
)
.await?;
// Since we waited for 'not_modified_since' to arrive, that is now the last
// record LSN. (Or close enough for our purposes; the last-record LSN can
// advance immediately after we return anyway)
Ok(not_modified_since)
} else {
// It might be better to use max(not_modified_since, latest_gc_cutoff_lsn)
@@ -2073,7 +2111,6 @@ impl PageServerHandler {
async fn handle_get_page_at_lsn_request_batched(
&mut self,
timeline: &Timeline,
effective_lsn: Lsn,
requests: smallvec::SmallVec<[BatchedGetPageRequest; 1]>,
io_concurrency: IoConcurrency,
ctx: &RequestContext,
@@ -2092,20 +2129,81 @@ impl PageServerHandler {
// Ignore error (trace buffer may be full or tracer may have disconnected).
_ = page_trace.try_send(PageTraceEvent {
key,
effective_lsn,
effective_lsn: batch.effective_request_lsn,
time,
});
}
}
// If any request in the batch needs to wait for LSN, then do so now.
let mut perf_instrument = false;
let max_effective_lsn = requests
.iter()
.map(|req| {
if req.ctx.has_perf_span() {
perf_instrument = true;
}
req.effective_request_lsn
})
.max()
.expect("batch is never empty");
let ctx = match perf_instrument {
true => RequestContextBuilder::from(ctx)
.root_perf_span(|| {
info_span!(
target: PERF_TRACE_TARGET,
"GET_VECTORED",
tenant_id = %timeline.tenant_shard_id.tenant_id,
timeline_id = %timeline.timeline_id,
shard = %timeline.tenant_shard_id.shard_slug(),
%max_effective_lsn
)
})
.attached_child(),
false => ctx.attached_child(),
};
let last_record_lsn = timeline.get_last_record_lsn();
if max_effective_lsn > last_record_lsn {
if let Err(e) = timeline
.wait_lsn(
max_effective_lsn,
crate::tenant::timeline::WaitLsnWaiter::PageService,
timeline::WaitLsnTimeout::Default,
&ctx,
)
.maybe_perf_instrument(&ctx, |current_perf_span| {
info_span!(
target: PERF_TRACE_TARGET,
parent: current_perf_span,
"WAIT_LSN",
)
})
.await
{
return Vec::from_iter(requests.into_iter().map(|req| {
Err(BatchedPageStreamError {
err: PageStreamError::from(e.clone()),
req: req.req.hdr,
})
}));
}
}
let results = timeline
.get_rel_page_at_lsn_batched(
requests
.iter()
.map(|p| (&p.req.rel, &p.req.blkno, p.ctx.attached_child())),
effective_lsn,
requests.iter().map(|p| {
(
&p.req.rel,
&p.req.blkno,
p.effective_request_lsn,
p.ctx.attached_child(),
)
}),
io_concurrency,
ctx,
&ctx,
)
.await;
assert_eq!(results.len(), requests.len());

View File

@@ -6,7 +6,7 @@
//! walingest.rs handles a few things like implicit relation creation and extension.
//! Clarify that)
//!
use std::collections::{BTreeMap, HashMap, HashSet, hash_map};
use std::collections::{HashMap, HashSet, hash_map};
use std::ops::{ControlFlow, Range};
use crate::walingest::{WalIngestError, WalIngestErrorKind};
@@ -14,7 +14,6 @@ use crate::{PERF_TRACE_TARGET, ensure_walingest};
use anyhow::Context;
use bytes::{Buf, Bytes, BytesMut};
use enum_map::Enum;
use itertools::Itertools;
use pageserver_api::key::{
AUX_FILES_KEY, CHECKPOINT_KEY, CONTROLFILE_KEY, CompactKey, DBDIR_KEY, Key, RelDirExists,
TWOPHASEDIR_KEY, dbdir_key_range, rel_block_to_key, rel_dir_to_key, rel_key_range,
@@ -22,7 +21,7 @@ use pageserver_api::key::{
repl_origin_key, repl_origin_key_range, slru_block_to_key, slru_dir_to_key,
slru_segment_key_range, slru_segment_size_to_key, twophase_file_key, twophase_key_range,
};
use pageserver_api::keyspace::SparseKeySpace;
use pageserver_api::keyspace::{KeySpaceRandomAccum, SparseKeySpace};
use pageserver_api::models::RelSizeMigration;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::{BlockNumber, RelTag, SlruKind};
@@ -41,7 +40,7 @@ use wal_decoder::serialized_batch::{SerializedValueBatch, ValueMeta};
use super::tenant::{PageReconstructError, Timeline};
use crate::aux_file;
use crate::context::{PerfInstrumentFutureExt, RequestContext, RequestContextBuilder};
use crate::context::{PerfInstrumentFutureExt, RequestContext};
use crate::keyspace::{KeySpace, KeySpaceAccum};
use crate::metrics::{
RELSIZE_CACHE_ENTRIES, RELSIZE_CACHE_HITS, RELSIZE_CACHE_MISSES, RELSIZE_CACHE_MISSES_OLD,
@@ -51,7 +50,7 @@ use crate::span::{
debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id,
};
use crate::tenant::storage_layer::IoConcurrency;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::timeline::{GetVectoredError, VersionedKeySpaceQuery};
/// Max delta records appended to the AUX_FILES_KEY (for aux v1). The write path will write a full image once this threshold is reached.
pub const MAX_AUX_FILE_DELTAS: usize = 1024;
@@ -207,10 +206,9 @@ impl Timeline {
let pages: smallvec::SmallVec<[_; 1]> = smallvec::smallvec![(tag, blknum)];
let res = self
.get_rel_page_at_lsn_batched(
pages
.iter()
.map(|(tag, blknum)| (tag, blknum, ctx.attached_child())),
effective_lsn,
pages.iter().map(|(tag, blknum)| {
(tag, blknum, effective_lsn, ctx.attached_child())
}),
io_concurrency.clone(),
ctx,
)
@@ -248,8 +246,7 @@ impl Timeline {
/// The ordering of the returned vec corresponds to the ordering of `pages`.
pub(crate) async fn get_rel_page_at_lsn_batched(
&self,
pages: impl ExactSizeIterator<Item = (&RelTag, &BlockNumber, RequestContext)>,
effective_lsn: Lsn,
pages: impl ExactSizeIterator<Item = (&RelTag, &BlockNumber, Lsn, RequestContext)>,
io_concurrency: IoConcurrency,
ctx: &RequestContext,
) -> Vec<Result<Bytes, PageReconstructError>> {
@@ -262,11 +259,13 @@ impl Timeline {
let mut result = Vec::with_capacity(pages.len());
let result_slots = result.spare_capacity_mut();
let mut keys_slots: BTreeMap<Key, smallvec::SmallVec<[(usize, RequestContext); 1]>> =
BTreeMap::default();
let mut keys_slots: HashMap<Key, smallvec::SmallVec<[(usize, RequestContext); 1]>> =
HashMap::with_capacity(pages.len());
let mut perf_instrument = false;
for (response_slot_idx, (tag, blknum, ctx)) in pages.enumerate() {
let mut req_keyspaces: HashMap<Lsn, KeySpaceRandomAccum> =
HashMap::with_capacity(pages.len());
for (response_slot_idx, (tag, blknum, lsn, ctx)) in pages.enumerate() {
if tag.relnode == 0 {
result_slots[response_slot_idx].write(Err(PageReconstructError::Other(
RelationError::InvalidRelnode.into(),
@@ -277,14 +276,14 @@ impl Timeline {
}
let nblocks = match self
.get_rel_size(*tag, Version::Lsn(effective_lsn), &ctx)
.get_rel_size(*tag, Version::Lsn(lsn), &ctx)
.maybe_perf_instrument(&ctx, |crnt_perf_span| {
info_span!(
target: PERF_TRACE_TARGET,
parent: crnt_perf_span,
"GET_REL_SIZE",
reltag=%tag,
lsn=%effective_lsn,
lsn=%lsn,
)
})
.await
@@ -300,7 +299,7 @@ impl Timeline {
if *blknum >= nblocks {
debug!(
"read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page",
tag, blknum, effective_lsn, nblocks
tag, blknum, lsn, nblocks
);
result_slots[response_slot_idx].write(Ok(ZERO_PAGE.clone()));
slots_filled += 1;
@@ -309,46 +308,29 @@ impl Timeline {
let key = rel_block_to_key(*tag, *blknum);
if ctx.has_perf_span() {
perf_instrument = true;
}
let key_slots = keys_slots.entry(key).or_default();
key_slots.push((response_slot_idx, ctx));
let acc = req_keyspaces.entry(lsn).or_default();
acc.add_key(key);
}
let keyspace = {
// add_key requires monotonicity
let mut acc = KeySpaceAccum::new();
for key in keys_slots
.keys()
// in fact it requires strong monotonicity
.dedup()
{
acc.add_key(*key);
}
acc.to_keyspace()
};
let ctx = match perf_instrument {
true => RequestContextBuilder::from(ctx)
.root_perf_span(|| {
info_span!(
target: PERF_TRACE_TARGET,
"GET_VECTORED",
tenant_id = %self.tenant_shard_id.tenant_id,
timeline_id = %self.timeline_id,
lsn = %effective_lsn,
shard = %self.tenant_shard_id.shard_slug(),
)
})
.attached_child(),
false => ctx.attached_child(),
};
let query: Vec<(Lsn, KeySpace)> = req_keyspaces
.into_iter()
.map(|(lsn, acc)| (lsn, acc.to_keyspace()))
.collect();
let query = VersionedKeySpaceQuery::scattered(query);
let res = self
.get_vectored(keyspace, effective_lsn, io_concurrency, &ctx)
.maybe_perf_instrument(&ctx, |current_perf_span| current_perf_span.clone())
.get_vectored(query, io_concurrency, ctx)
.maybe_perf_instrument(ctx, |current_perf_span| {
info_span!(
target: PERF_TRACE_TARGET,
parent: current_perf_span,
"GET_BATCH",
batch_size = %page_count,
)
})
.await;
match res {
@@ -378,12 +360,12 @@ impl Timeline {
// There is no standardized way to express that the batched span followed from N request spans.
// So, abuse the system and mark the request contexts as follows_from the batch span, so we get
// some linkage in our trace viewer. It allows us to answer: which GET_VECTORED did this GET_PAGE wait for.
req_ctx.perf_follows_from(&ctx);
req_ctx.perf_follows_from(ctx);
slots_filled += 1;
}
result_slots[first_slot].write(res);
first_req_ctx.perf_follows_from(&ctx);
first_req_ctx.perf_follows_from(ctx);
slots_filled += 1;
}
}
@@ -422,7 +404,7 @@ impl Timeline {
}
};
req_ctx.perf_follows_from(&ctx);
req_ctx.perf_follows_from(ctx);
result_slots[*slot].write(err);
}
@@ -661,8 +643,9 @@ impl Timeline {
let mut segment = BytesMut::with_capacity(n_blocks as usize * BLCKSZ as usize);
for batch in batches.parts {
let query = VersionedKeySpaceQuery::uniform(batch, lsn);
let blocks = self
.get_vectored(batch, lsn, io_concurrency.clone(), ctx)
.get_vectored(query, io_concurrency.clone(), ctx)
.await?;
for (_key, block) in blocks {
@@ -899,8 +882,9 @@ impl Timeline {
);
for batch in batches.parts.into_iter().rev() {
let query = VersionedKeySpaceQuery::uniform(batch, probe_lsn);
let blocks = self
.get_vectored(batch, probe_lsn, io_concurrency.clone(), ctx)
.get_vectored(query, io_concurrency.clone(), ctx)
.await?;
for (_key, clog_page) in blocks.into_iter().rev() {

View File

@@ -5948,7 +5948,7 @@ mod tests {
use timeline::InMemoryLayerTestDesc;
#[cfg(feature = "testing")]
use timeline::compaction::{KeyHistoryRetention, KeyLogAtLsn};
use timeline::{CompactOptions, DeltaLayerTestDesc};
use timeline::{CompactOptions, DeltaLayerTestDesc, VersionedKeySpaceQuery};
use utils::id::TenantId;
use super::*;
@@ -6786,10 +6786,11 @@ mod tests {
for read in reads {
info!("Doing vectored read on {:?}", read);
let query = VersionedKeySpaceQuery::uniform(read.clone(), reads_lsn);
let vectored_res = tline
.get_vectored_impl(
read.clone(),
reads_lsn,
query,
&mut ValuesReconstructState::new(io_concurrency.clone()),
&ctx,
)
@@ -6868,10 +6869,11 @@ mod tests {
};
let read_lsn = child_timeline.get_last_record_lsn();
let query = VersionedKeySpaceQuery::uniform(aux_keyspace.clone(), read_lsn);
let vectored_res = child_timeline
.get_vectored_impl(
aux_keyspace.clone(),
read_lsn,
query,
&mut ValuesReconstructState::new(io_concurrency.clone()),
&ctx,
)
@@ -7017,10 +7019,12 @@ mod tests {
let read = KeySpace {
ranges: vec![key_near_gap..gap_at_key.next(), key_near_end..current_key],
};
let query = VersionedKeySpaceQuery::uniform(read.clone(), current_lsn);
let results = child_timeline
.get_vectored_impl(
read.clone(),
current_lsn,
query,
&mut ValuesReconstructState::new(io_concurrency.clone()),
&ctx,
)
@@ -7151,12 +7155,16 @@ mod tests {
}
for query_lsn in query_lsns {
let query = VersionedKeySpaceQuery::uniform(
KeySpace {
ranges: vec![child_gap_at_key..child_gap_at_key.next()],
},
query_lsn,
);
let results = child_timeline
.get_vectored_impl(
KeySpace {
ranges: vec![child_gap_at_key..child_gap_at_key.next()],
},
query_lsn,
query,
&mut ValuesReconstructState::new(io_concurrency.clone()),
&ctx,
)
@@ -7655,10 +7663,11 @@ mod tests {
}
let mut cnt = 0;
let query = VersionedKeySpaceQuery::uniform(keyspace.clone(), lsn);
for (key, value) in tline
.get_vectored_impl(
keyspace.clone(),
lsn,
query,
&mut ValuesReconstructState::new(io_concurrency.clone()),
&ctx,
)
@@ -7865,8 +7874,9 @@ mod tests {
io_concurrency: IoConcurrency,
) -> anyhow::Result<(BTreeMap<Key, Result<Bytes, PageReconstructError>>, usize)> {
let mut reconstruct_state = ValuesReconstructState::new(io_concurrency);
let query = VersionedKeySpaceQuery::uniform(keyspace.clone(), lsn);
let res = tline
.get_vectored_impl(keyspace.clone(), lsn, &mut reconstruct_state, ctx)
.get_vectored_impl(query, &mut reconstruct_state, ctx)
.await?;
Ok((res, reconstruct_state.get_delta_layers_visited() as usize))
}
@@ -8163,13 +8173,10 @@ mod tests {
// test vectored scan on parent timeline
let mut reconstruct_state = ValuesReconstructState::new(io_concurrency.clone());
let query =
VersionedKeySpaceQuery::uniform(KeySpace::single(Key::metadata_key_range()), lsn);
let res = tline
.get_vectored_impl(
KeySpace::single(Key::metadata_key_range()),
lsn,
&mut reconstruct_state,
&ctx,
)
.get_vectored_impl(query, &mut reconstruct_state, &ctx)
.await?;
assert_eq!(
@@ -8189,13 +8196,10 @@ mod tests {
// test vectored scan on child timeline
let mut reconstruct_state = ValuesReconstructState::new(io_concurrency.clone());
let query =
VersionedKeySpaceQuery::uniform(KeySpace::single(Key::metadata_key_range()), lsn);
let res = child
.get_vectored_impl(
KeySpace::single(Key::metadata_key_range()),
lsn,
&mut reconstruct_state,
&ctx,
)
.get_vectored_impl(query, &mut reconstruct_state, &ctx)
.await?;
assert_eq!(
@@ -8229,13 +8233,9 @@ mod tests {
let io_concurrency =
IoConcurrency::spawn_from_conf(tline.conf, tline.gate.enter().unwrap());
let mut reconstruct_state = ValuesReconstructState::new(io_concurrency);
let query = VersionedKeySpaceQuery::uniform(KeySpace::single(key..key.next()), lsn);
let mut res = tline
.get_vectored_impl(
KeySpace::single(key..key.next()),
lsn,
&mut reconstruct_state,
ctx,
)
.get_vectored_impl(query, &mut reconstruct_state, ctx)
.await?;
Ok(res.pop_last().map(|(k, v)| {
assert_eq!(k, key);
@@ -10369,14 +10369,13 @@ mod tests {
)
.await?;
let keyspace = KeySpace::single(get_key(0)..get_key(10));
let query = VersionedKeySpaceQuery::uniform(
KeySpace::single(get_key(0)..get_key(10)),
delta_layer_end_lsn,
);
let results = tline
.get_vectored(
keyspace,
delta_layer_end_lsn,
IoConcurrency::sequential(),
&ctx,
)
.get_vectored(query, IoConcurrency::sequential(), &ctx)
.await
.expect("No vectored errors");
for (key, res) in results {
@@ -10524,9 +10523,13 @@ mod tests {
)
.await?;
let keyspace = KeySpace::single(get_key(0)..get_key(10));
let query = VersionedKeySpaceQuery::uniform(
KeySpace::single(get_key(0)..get_key(10)),
last_record_lsn,
);
let results = tline
.get_vectored(keyspace, last_record_lsn, IoConcurrency::sequential(), &ctx)
.get_vectored(query, IoConcurrency::sequential(), &ctx)
.await
.expect("No vectored errors");
for (key, res) in results {

View File

@@ -715,13 +715,34 @@ pub(crate) enum LayerId {
}
/// Uniquely identify a layer visit by the layer
/// and LSN floor (or start LSN) of the reads.
/// The layer itself is not enough since we may
/// have different LSN lower bounds for delta layer reads.
/// and LSN range of the reads. Note that the end of the range is exclusive.
///
/// The layer itself is not enough since we may have different LSN lower
/// bounds for delta layer reads. Scenarios where this can happen are:
///
/// 1. Layer overlaps: imagine an image layer inside and in-memory layer
/// and a query that only partially hits the image layer. Part of the query
/// needs to read the whole in-memory layer and the other part needs to read
/// only up to the image layer. Hence, they'll have different LSN floor values
/// for the read.
///
/// 2. Scattered reads: the read path supports starting at different LSNs. Imagine
/// The start LSN for one range is inside a layer and the start LSN for another range
/// Is above the layer (includes all of it). Both ranges need to read the layer all the
/// Way to the end but starting at different points. Hence, they'll have different LSN
/// Ceil values.
///
/// The implication is that we might visit the same layer multiple times
/// in order to read different LSN ranges from it. In practice, this isn't very concerning
/// because:
/// 1. Layer overlaps are rare and generally not intended
/// 2. Scattered reads will stabilise after the first few layers provided their starting LSNs
/// are grouped tightly enough (likely the case).
#[derive(Debug, PartialEq, Eq, Clone, Hash)]
struct LayerToVisitId {
layer_id: LayerId,
lsn_floor: Lsn,
lsn_ceil: Lsn,
}
#[derive(Debug, PartialEq, Eq, Hash)]
@@ -805,6 +826,7 @@ impl LayerFringe {
let layer_to_visit_id = LayerToVisitId {
layer_id: layer.id(),
lsn_floor: lsn_range.start,
lsn_ceil: lsn_range.end,
};
let entry = self.visit_reads.entry(layer_to_visit_id.clone());

View File

@@ -585,7 +585,7 @@ pub(crate) enum PageReconstructError {
WalRedo(anyhow::Error),
#[error("{0}")]
MissingKey(MissingKeyError),
MissingKey(Box<MissingKeyError>),
}
impl From<anyhow::Error> for PageReconstructError {
@@ -690,16 +690,23 @@ impl std::fmt::Display for ReadPath {
#[derive(thiserror::Error)]
pub struct MissingKeyError {
key: Key,
keyspace: KeySpace,
shard: ShardNumber,
cont_lsn: Lsn,
request_lsn: Lsn,
query: Option<VersionedKeySpaceQuery>,
// This is largest request LSN from the get page request batch
original_hwm_lsn: Lsn,
ancestor_lsn: Option<Lsn>,
/// Debug information about the read path if there's an error
read_path: Option<ReadPath>,
backtrace: Option<std::backtrace::Backtrace>,
}
impl MissingKeyError {
fn enrich(&mut self, query: VersionedKeySpaceQuery) {
self.query = Some(query);
}
}
impl std::fmt::Debug for MissingKeyError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{}", self)
@@ -710,14 +717,18 @@ impl std::fmt::Display for MissingKeyError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"could not find data for key {} (shard {:?}) at LSN {}, request LSN {}",
self.key, self.shard, self.cont_lsn, self.request_lsn
"could not find data for key {} (shard {:?}), original HWM LSN {}",
self.keyspace, self.shard, self.original_hwm_lsn
)?;
if let Some(ref ancestor_lsn) = self.ancestor_lsn {
write!(f, ", ancestor {}", ancestor_lsn)?;
}
if let Some(ref query) = self.query {
write!(f, ", query {}", query)?;
}
if let Some(ref read_path) = self.read_path {
write!(f, "\n{}", read_path)?;
}
@@ -817,7 +828,7 @@ pub(crate) enum GetVectoredError {
InvalidLsn(Lsn),
#[error("requested key not found: {0}")]
MissingKey(MissingKeyError),
MissingKey(Box<MissingKeyError>),
#[error("ancestry walk")]
GetReadyAncestorError(#[source] GetReadyAncestorError),
@@ -928,7 +939,7 @@ impl std::fmt::Debug for Timeline {
}
}
#[derive(thiserror::Error, Debug)]
#[derive(thiserror::Error, Debug, Clone)]
pub(crate) enum WaitLsnError {
// Called on a timeline which is shutting down
#[error("Shutdown")]
@@ -1128,14 +1139,12 @@ impl Timeline {
// page_service.
debug_assert!(!self.shard_identity.is_key_disposable(&key));
let keyspace = KeySpace {
ranges: vec![key..key.next()],
};
let mut reconstruct_state = ValuesReconstructState::new(IoConcurrency::sequential());
let query = VersionedKeySpaceQuery::uniform(KeySpace::single(key..key.next()), lsn);
let vectored_res = self
.get_vectored_impl(keyspace.clone(), lsn, &mut reconstruct_state, ctx)
.get_vectored_impl(query, &mut reconstruct_state, ctx)
.await;
let key_value = vectored_res?.pop_first();
@@ -1153,15 +1162,17 @@ impl Timeline {
value
}
}
None => Err(PageReconstructError::MissingKey(MissingKeyError {
key,
shard: self.shard_identity.get_shard_number(&key),
cont_lsn: Lsn(0),
request_lsn: lsn,
ancestor_lsn: None,
backtrace: None,
read_path: None,
})),
None => Err(PageReconstructError::MissingKey(Box::new(
MissingKeyError {
keyspace: KeySpace::single(key..key.next()),
shard: self.shard_identity.get_shard_number(&key),
original_hwm_lsn: lsn,
ancestor_lsn: None,
backtrace: None,
read_path: None,
query: None,
},
))),
}
}
@@ -1174,21 +1185,18 @@ impl Timeline {
/// which actually vectorizes the read path.
pub(crate) async fn get_vectored(
&self,
keyspace: KeySpace,
lsn: Lsn,
query: VersionedKeySpaceQuery,
io_concurrency: super::storage_layer::IoConcurrency,
ctx: &RequestContext,
) -> Result<BTreeMap<Key, Result<Bytes, PageReconstructError>>, GetVectoredError> {
if !lsn.is_valid() {
return Err(GetVectoredError::InvalidLsn(lsn));
}
let total_keyspace = query.total_keyspace();
let key_count = keyspace.total_raw_size().try_into().unwrap();
let key_count = total_keyspace.total_raw_size().try_into().unwrap();
if key_count > Timeline::MAX_GET_VECTORED_KEYS {
return Err(GetVectoredError::Oversized(key_count));
}
for range in &keyspace.ranges {
for range in &total_keyspace.ranges {
let mut key = range.start;
while key != range.end {
assert!(!self.shard_identity.is_key_disposable(&key));
@@ -1197,9 +1205,8 @@ impl Timeline {
}
trace!(
"get vectored request for {:?}@{} from task kind {:?}",
keyspace,
lsn,
"get vectored query {} from task kind {:?}",
query,
ctx.task_kind(),
);
@@ -1208,12 +1215,7 @@ impl Timeline {
.map(|metric| (metric, Instant::now()));
let res = self
.get_vectored_impl(
keyspace.clone(),
lsn,
&mut ValuesReconstructState::new(io_concurrency),
ctx,
)
.get_vectored_impl(query, &mut ValuesReconstructState::new(io_concurrency), ctx)
.await;
if let Some((metric, start)) = start {
@@ -1264,13 +1266,10 @@ impl Timeline {
.for_task_kind(ctx.task_kind())
.map(ScanLatencyOngoingRecording::start_recording);
let query = VersionedKeySpaceQuery::uniform(keyspace, lsn);
let vectored_res = self
.get_vectored_impl(
keyspace.clone(),
lsn,
&mut ValuesReconstructState::new(io_concurrency),
ctx,
)
.get_vectored_impl(query, &mut ValuesReconstructState::new(io_concurrency), ctx)
.await;
if let Some(recording) = start {
@@ -1282,16 +1281,19 @@ impl Timeline {
pub(super) async fn get_vectored_impl(
&self,
keyspace: KeySpace,
lsn: Lsn,
query: VersionedKeySpaceQuery,
reconstruct_state: &mut ValuesReconstructState,
ctx: &RequestContext,
) -> Result<BTreeMap<Key, Result<Bytes, PageReconstructError>>, GetVectoredError> {
let read_path = if self.conf.enable_read_path_debugging || ctx.read_path_debug() {
Some(ReadPath::new(keyspace.clone(), lsn))
Some(ReadPath::new(
query.total_keyspace(),
query.high_watermark_lsn()?,
))
} else {
None
};
reconstruct_state.read_path = read_path;
let redo_attempt_type = if ctx.task_kind() == TaskKind::Compaction {
@@ -1311,7 +1313,7 @@ impl Timeline {
})
.attached_child();
self.get_vectored_reconstruct_data(keyspace.clone(), lsn, reconstruct_state, &ctx)
self.get_vectored_reconstruct_data(query.clone(), reconstruct_state, &ctx)
.maybe_perf_instrument(&ctx, |crnt_perf_span| crnt_perf_span.clone())
.await
};
@@ -1324,6 +1326,13 @@ impl Timeline {
.map(|state| state.collect_pending_ios())
.collect::<FuturesUnordered<_>>();
while collect_futs.next().await.is_some() {}
// Enrich the missing key error with the original query.
if let GetVectoredError::MissingKey(mut missing_err) = err {
missing_err.enrich(query.clone());
return Err(GetVectoredError::MissingKey(missing_err));
}
return Err(err);
};
@@ -1341,6 +1350,8 @@ impl Timeline {
let futs = FuturesUnordered::new();
for (key, state) in std::mem::take(&mut reconstruct_state.keys) {
let req_lsn_for_key = query.map_key_to_lsn(&key);
futs.push({
let walredo_self = self.myself.upgrade().expect("&self method holds the arc");
let ctx = RequestContextBuilder::from(&ctx)
@@ -1387,7 +1398,7 @@ impl Timeline {
let walredo_deltas = converted.num_deltas();
let walredo_res = walredo_self
.reconstruct_value(key, lsn, converted, redo_attempt_type)
.reconstruct_value(key, req_lsn_for_key, converted, redo_attempt_type)
.maybe_perf_instrument(&ctx, |crnt_perf_span| {
info_span!(
target: PERF_TRACE_TARGET,
@@ -1414,15 +1425,18 @@ impl Timeline {
// to avoid infinite results.
if !results.is_empty() {
if layers_visited >= Self::LAYERS_VISITED_WARN_THRESHOLD {
let total_keyspace = query.total_keyspace();
let max_request_lsn = query.high_watermark_lsn().expect("Validated previously");
static LOG_PACER: Lazy<Mutex<RateLimit>> =
Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(60))));
LOG_PACER.lock().unwrap().call(|| {
let num_keys = keyspace.total_raw_size();
let num_keys = total_keyspace.total_raw_size();
let num_pages = results.len();
tracing::info!(
shard_id = %self.tenant_shard_id.shard_slug(),
lsn = %lsn,
"Vectored read for {keyspace} visited {layers_visited} layers. Returned {num_pages}/{num_keys} pages.",
lsn = %max_request_lsn,
"Vectored read for {total_keyspace} visited {layers_visited} layers. Returned {num_pages}/{num_keys} pages.",
);
});
}
@@ -3940,6 +3954,154 @@ impl Timeline {
}
}
#[derive(Clone)]
/// Type representing a query in the ([`Lsn`], [`Key`]) space.
/// In other words, a set of segments in a 2D space.
///
/// This representation has the advatange of avoiding hash map
/// allocations for uniform queries.
pub(crate) enum VersionedKeySpaceQuery {
/// Variant for queries at a single [`Lsn`]
Uniform { keyspace: KeySpace, lsn: Lsn },
/// Variant for queries at multiple [`Lsn`]s
Scattered {
keyspaces_at_lsn: Vec<(Lsn, KeySpace)>,
},
}
impl VersionedKeySpaceQuery {
pub(crate) fn uniform(keyspace: KeySpace, lsn: Lsn) -> Self {
Self::Uniform { keyspace, lsn }
}
pub(crate) fn scattered(keyspaces_at_lsn: Vec<(Lsn, KeySpace)>) -> Self {
Self::Scattered { keyspaces_at_lsn }
}
/// Returns the most recent (largest) LSN included in the query.
/// If any of the LSNs included in the query are invalid, returns
/// an error instead.
fn high_watermark_lsn(&self) -> Result<Lsn, GetVectoredError> {
match self {
Self::Uniform { lsn, .. } => {
if !lsn.is_valid() {
return Err(GetVectoredError::InvalidLsn(*lsn));
}
Ok(*lsn)
}
Self::Scattered { keyspaces_at_lsn } => {
let mut max_lsn = None;
for (lsn, _keyspace) in keyspaces_at_lsn.iter() {
if !lsn.is_valid() {
return Err(GetVectoredError::InvalidLsn(*lsn));
}
max_lsn = std::cmp::max(max_lsn, Some(lsn));
}
if let Some(computed) = max_lsn {
Ok(*computed)
} else {
Err(GetVectoredError::Other(anyhow!("empty input")))
}
}
}
}
/// Returns the total keyspace being queried: the result of projecting
/// everything in the key dimensions onto the key axis.
fn total_keyspace(&self) -> KeySpace {
match self {
Self::Uniform { keyspace, .. } => keyspace.clone(),
Self::Scattered { keyspaces_at_lsn } => keyspaces_at_lsn
.iter()
.map(|(_lsn, keyspace)| keyspace)
.fold(KeySpace::default(), |mut acc, v| {
acc.merge(v);
acc
}),
}
}
/// Returns LSN for a specific key.
///
/// Invariant: requested key must be part of [`Self::total_keyspace`]
fn map_key_to_lsn(&self, key: &Key) -> Lsn {
match self {
Self::Uniform { lsn, .. } => *lsn,
Self::Scattered { keyspaces_at_lsn } => {
keyspaces_at_lsn
.iter()
.find(|(_lsn, keyspace)| keyspace.contains(key))
.expect("Returned key was requested")
.0
}
}
}
/// Remove any parts of the query (segments) which overlap with the provided
/// key space (also segments).
fn remove_overlapping_with(&mut self, to_remove: &KeySpace) -> KeySpace {
match self {
Self::Uniform { keyspace, .. } => keyspace.remove_overlapping_with(to_remove),
Self::Scattered { keyspaces_at_lsn } => {
let mut removed_accum = KeySpaceRandomAccum::new();
keyspaces_at_lsn.iter_mut().for_each(|(_lsn, keyspace)| {
let removed = keyspace.remove_overlapping_with(to_remove);
removed_accum.add_keyspace(removed);
});
removed_accum.to_keyspace()
}
}
}
fn is_empty(&self) -> bool {
match self {
Self::Uniform { keyspace, .. } => keyspace.is_empty(),
Self::Scattered { keyspaces_at_lsn } => keyspaces_at_lsn
.iter()
.all(|(_lsn, keyspace)| keyspace.is_empty()),
}
}
/// "Lower" the query on the LSN dimension
fn lower(&mut self, to: Lsn) {
match self {
Self::Uniform { lsn, .. } => {
// If the originally requested LSN is smaller than the starting
// LSN of the ancestor we are descending into, we need to respect that.
// Hence the min.
*lsn = std::cmp::min(*lsn, to);
}
Self::Scattered { keyspaces_at_lsn } => {
keyspaces_at_lsn.iter_mut().for_each(|(lsn, _keyspace)| {
*lsn = std::cmp::min(*lsn, to);
});
}
}
}
}
impl std::fmt::Display for VersionedKeySpaceQuery {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "[")?;
match self {
VersionedKeySpaceQuery::Uniform { keyspace, lsn } => {
write!(f, "{keyspace} @ {lsn}")?;
}
VersionedKeySpaceQuery::Scattered { keyspaces_at_lsn } => {
for (lsn, keyspace) in keyspaces_at_lsn.iter() {
write!(f, "{keyspace} @ {lsn},")?;
}
}
}
write!(f, "]")
}
}
impl Timeline {
#[allow(clippy::doc_lazy_continuation)]
/// Get the data needed to reconstruct all keys in the provided keyspace
@@ -3954,16 +4116,15 @@ impl Timeline {
/// 2.4. If the fringe is empty, go back to 1
async fn get_vectored_reconstruct_data(
&self,
mut keyspace: KeySpace,
request_lsn: Lsn,
mut query: VersionedKeySpaceQuery,
reconstruct_state: &mut ValuesReconstructState,
ctx: &RequestContext,
) -> Result<(), GetVectoredError> {
let original_hwm_lsn = query.high_watermark_lsn().unwrap();
let mut timeline_owned: Arc<Timeline>;
let mut timeline = self;
let mut cont_lsn = Lsn(request_lsn.0 + 1);
let missing_keyspace = loop {
if self.cancel.is_cancelled() {
return Err(GetVectoredError::Cancelled);
@@ -3980,15 +4141,14 @@ impl Timeline {
parent: crnt_perf_span,
"PLAN_IO_TIMELINE",
timeline = %timeline.timeline_id,
lsn = %cont_lsn,
high_watermark_lsn = %query.high_watermark_lsn().unwrap(),
)
})
.attached_child();
Self::get_vectored_reconstruct_data_timeline(
timeline,
keyspace.clone(),
cont_lsn,
&query,
reconstruct_state,
&self.cancel,
&ctx,
@@ -3997,23 +4157,23 @@ impl Timeline {
.await?
};
keyspace.remove_overlapping_with(&completed);
query.remove_overlapping_with(&completed);
// Do not descend into the ancestor timeline for aux files.
// We don't return a blanket [`GetVectoredError::MissingKey`] to avoid
// stalling compaction.
keyspace.remove_overlapping_with(&KeySpace {
query.remove_overlapping_with(&KeySpace {
ranges: vec![NON_INHERITED_RANGE, Key::sparse_non_inherited_keyspace()],
});
// Keyspace is fully retrieved
if keyspace.is_empty() {
if query.is_empty() {
break None;
}
let Some(ancestor_timeline) = timeline.ancestor_timeline.as_ref() else {
// Not fully retrieved but no ancestor timeline.
break Some(keyspace);
break Some(query.total_keyspace());
};
// Now we see if there are keys covered by the image layer but does not exist in the
@@ -4024,7 +4184,7 @@ impl Timeline {
// keys from `keyspace`, we expect there to be no overlap between it and the image covered key
// space. If that's not the case, we had at least one key encounter a gap in the image layer
// and stop the search as a result of that.
let mut removed = keyspace.remove_overlapping_with(&image_covered_keyspace);
let mut removed = query.remove_overlapping_with(&image_covered_keyspace);
// Do not fire missing key error and end early for sparse keys. Note that we hava already removed
// non-inherited keyspaces before, so we can safely do a full `SPARSE_RANGE` remove instead of
// figuring out what is the inherited key range and do a fine-grained pruning.
@@ -4034,11 +4194,11 @@ impl Timeline {
if !removed.is_empty() {
break Some(removed);
}
// If we reached this point, `remove_overlapping_with` should not have made any change to the
// keyspace.
// Take the min to avoid reconstructing a page with data newer than request Lsn.
cont_lsn = std::cmp::min(Lsn(request_lsn.0 + 1), Lsn(timeline.ancestor_lsn.0 + 1));
// Each key range in the original query is at some point in the LSN space.
// When descending into the ancestor, lower all ranges in the LSN space
// such that new changes on the parent timeline are not visible.
query.lower(timeline.ancestor_lsn);
let ctx = RequestContextBuilder::from(ctx)
.perf_span(|crnt_perf_span| {
@@ -4047,7 +4207,6 @@ impl Timeline {
parent: crnt_perf_span,
"GET_ANCESTOR",
timeline = %timeline.timeline_id,
lsn = %cont_lsn,
ancestor = %ancestor_timeline.timeline_id,
ancestor_lsn = %timeline.ancestor_lsn
)
@@ -4077,22 +4236,47 @@ impl Timeline {
};
if let Some(missing_keyspace) = missing_keyspace {
return Err(GetVectoredError::MissingKey(MissingKeyError {
key: missing_keyspace.start().unwrap(), /* better if we can store the full keyspace */
shard: self
.shard_identity
.get_shard_number(&missing_keyspace.start().unwrap()),
cont_lsn,
request_lsn,
return Err(GetVectoredError::MissingKey(Box::new(MissingKeyError {
keyspace: missing_keyspace, /* better if we can store the full keyspace */
shard: self.shard_identity.number,
original_hwm_lsn,
ancestor_lsn: Some(timeline.ancestor_lsn),
backtrace: None,
read_path: std::mem::take(&mut reconstruct_state.read_path),
}));
query: None,
})));
}
Ok(())
}
async fn get_vectored_init_fringe(
&self,
query: &VersionedKeySpaceQuery,
) -> Result<LayerFringe, GetVectoredError> {
let mut fringe = LayerFringe::new();
let guard = self.layers.read().await;
match query {
VersionedKeySpaceQuery::Uniform { keyspace, lsn } => {
// LSNs requested by the compute or determined by the pageserver
// are inclusive. Queries to the layer map use exclusive LSNs.
// Hence, bump the value before the query - same in the other
// match arm.
let cont_lsn = Lsn(lsn.0 + 1);
guard.update_search_fringe(keyspace, cont_lsn, &mut fringe)?;
}
VersionedKeySpaceQuery::Scattered { keyspaces_at_lsn } => {
for (lsn, keyspace) in keyspaces_at_lsn.iter() {
let cont_lsn_for_keyspace = Lsn(lsn.0 + 1);
guard.update_search_fringe(keyspace, cont_lsn_for_keyspace, &mut fringe)?;
}
}
}
Ok(fringe)
}
/// Collect the reconstruct data for a keyspace from the specified timeline.
///
/// Maintain a fringe [`LayerFringe`] which tracks all the layers that intersect
@@ -4111,8 +4295,7 @@ impl Timeline {
/// decides how to deal with these two keyspaces.
async fn get_vectored_reconstruct_data_timeline(
timeline: &Timeline,
keyspace: KeySpace,
mut cont_lsn: Lsn,
query: &VersionedKeySpaceQuery,
reconstruct_state: &mut ValuesReconstructState,
cancel: &CancellationToken,
ctx: &RequestContext,
@@ -4128,14 +4311,7 @@ impl Timeline {
let _guard = timeline.gc_compaction_layer_update_lock.read().await;
// Initialize the fringe
let mut fringe = {
let mut fringe = LayerFringe::new();
let guard = timeline.layers.read().await;
guard.update_search_fringe(&keyspace, cont_lsn, &mut fringe)?;
fringe
};
let mut fringe = timeline.get_vectored_init_fringe(query).await?;
let mut completed_keyspace = KeySpace::default();
let mut image_covered_keyspace = KeySpaceRandomAccum::new();
@@ -4161,7 +4337,7 @@ impl Timeline {
.await?;
let mut unmapped_keyspace = keyspace_to_read;
cont_lsn = next_cont_lsn;
let cont_lsn = next_cont_lsn;
reconstruct_state.on_layer_visited(&layer_to_read);
@@ -4996,13 +5172,11 @@ impl Timeline {
if key_request_accum.raw_size() >= Timeline::MAX_GET_VECTORED_KEYS
|| (last_key_in_range && key_request_accum.raw_size() > 0)
{
let query =
VersionedKeySpaceQuery::uniform(key_request_accum.consume_keyspace(), lsn);
let results = self
.get_vectored(
key_request_accum.consume_keyspace(),
lsn,
io_concurrency.clone(),
ctx,
)
.get_vectored(query, io_concurrency.clone(), ctx)
.await?;
if self.cancel.is_cancelled() {
@@ -5091,7 +5265,11 @@ impl Timeline {
// Directly use `get_vectored_impl` to skip the max_vectored_read_key limit check. Note that the keyspace should
// not contain too many keys, otherwise this takes a lot of memory.
let data = self
.get_vectored_impl(partition.clone(), lsn, &mut reconstruct_state, ctx)
.get_vectored_impl(
VersionedKeySpaceQuery::uniform(partition.clone(), lsn),
&mut reconstruct_state,
ctx,
)
.await?;
let (data, total_kb_retrieved, total_keys_retrieved) = {
let mut new_data = BTreeMap::new();

View File

@@ -30,6 +30,7 @@ use crate::tenant::storage_layer::{
AsLayerDesc as _, DeltaLayerWriter, ImageLayerWriter, IoConcurrency, Layer, ResidentLayer,
ValuesReconstructState,
};
use crate::tenant::timeline::VersionedKeySpaceQuery;
use crate::virtual_file::{MaybeFatalIo, VirtualFile};
#[derive(Debug, thiserror::Error)]
@@ -212,13 +213,9 @@ async fn generate_tombstone_image_layer(
}
}
let query = VersionedKeySpaceQuery::uniform(KeySpace::single(key_range.clone()), image_lsn);
let data = ancestor
.get_vectored_impl(
KeySpace::single(key_range.clone()),
image_lsn,
&mut reconstruct_state,
ctx,
)
.get_vectored_impl(query, &mut reconstruct_state, ctx)
.await
.context("failed to retrieve aux keys")
.map_err(|e| Error::launder(e, Error::Prepare))?;

View File

@@ -1255,6 +1255,7 @@ class NeonEnv:
"mode": "pipelined",
"execution": "concurrent-futures",
"max_batch_size": 32,
"batching": "scattered-lsn",
}
get_vectored_concurrent_io = self.pageserver_get_vectored_concurrent_io
@@ -1321,6 +1322,10 @@ class NeonEnv:
log.info("test may use old binaries, ignoring warnings about unknown config items")
ps.allowed_errors.append(".*ignoring unknown configuration item.*")
# Allow old software to start until https://github.com/neondatabase/neon/pull/11275
# lands in the compatiblity snapshot.
ps_cfg["page_service_pipelining"].pop("batching")
self.pageservers.append(ps)
cfg["pageservers"].append(ps_cfg)

View File

@@ -31,20 +31,28 @@ class PageServicePipeliningConfigSerial(PageServicePipeliningConfig):
class PageServicePipeliningConfigPipelined(PageServicePipeliningConfig):
max_batch_size: int
execution: str
batching: str
mode: str = "pipelined"
EXECUTION = ["concurrent-futures"]
BATCHING = ["uniform-lsn", "scattered-lsn"]
NON_BATCHABLE: list[PageServicePipeliningConfig] = [PageServicePipeliningConfigSerial()]
for max_batch_size in [1, 32]:
for execution in EXECUTION:
NON_BATCHABLE.append(PageServicePipeliningConfigPipelined(max_batch_size, execution))
for batching in BATCHING:
NON_BATCHABLE.append(
PageServicePipeliningConfigPipelined(max_batch_size, execution, batching)
)
BATCHABLE: list[PageServicePipeliningConfig] = []
for max_batch_size in [32]:
for execution in EXECUTION:
BATCHABLE.append(PageServicePipeliningConfigPipelined(max_batch_size, execution))
for batching in BATCHING:
BATCHABLE.append(
PageServicePipeliningConfigPipelined(max_batch_size, execution, batching)
)
@pytest.mark.parametrize(
@@ -300,7 +308,10 @@ def test_throughput(
PRECISION_CONFIGS: list[PageServicePipeliningConfig] = [PageServicePipeliningConfigSerial()]
for max_batch_size in [1, 32]:
for execution in EXECUTION:
PRECISION_CONFIGS.append(PageServicePipeliningConfigPipelined(max_batch_size, execution))
for batching in BATCHING:
PRECISION_CONFIGS.append(
PageServicePipeliningConfigPipelined(max_batch_size, execution, batching)
)
@pytest.mark.parametrize(

View File

@@ -16,6 +16,7 @@ def test_slow_flush(neon_env_builder: NeonEnvBuilder, neon_binpath: Path, kind:
"mode": "pipelined",
"max_batch_size": 32,
"execution": "concurrent-futures",
"batching": "uniform-lsn",
}
neon_env_builder.pageserver_config_override = patch_pageserver_toml