Compare commits

...

6 Commits

Author SHA1 Message Date
John Spray
9c097aa75f Revised fragmentation logic 2024-04-25 18:15:26 +01:00
John Spray
1330a60d27 pull key diff into helper 2024-04-25 18:05:59 +01:00
John Spray
5c1135afcd tests: add test_sharding_compaction 2024-04-25 18:05:59 +01:00
John Spray
4316f0fab2 Tidy up old size code 2024-04-25 18:05:59 +01:00
John Spray
987bfa23e1 pageserver: use shard-aware partitioning in tiered compaction 2024-04-25 18:04:23 +01:00
John Spray
43ec37adf6 pageserver: shard-aware keyspace partitioning 2024-04-25 18:04:23 +01:00
11 changed files with 650 additions and 82 deletions

View File

@@ -1,7 +1,10 @@
use postgres_ffi::BLCKSZ;
use std::ops::Range;
use crate::key::Key;
use crate::{
key::Key,
shard::{ShardCount, ShardIdentity},
};
use itertools::Itertools;
///
@@ -14,44 +17,246 @@ pub struct KeySpace {
pub ranges: Vec<Range<Key>>,
}
/// Represents a contiguous half-open range of the keyspace, masked according to a particular
/// ShardNumber's stripes: within this range of keys, only some "belong" to the current
/// shard.
///
/// When we iterate over keys within this object, we will skip any keys that don't belong
/// to this shard.
///
/// The start + end keys may not belong to the shard: these specify where layer files should
/// start + end, but we will never actually read/write those keys.
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct ShardedRange<'a> {
pub shard_identity: &'a ShardIdentity,
pub range: Range<Key>,
}
// Calculate the distance between two keys, assuming that they are somewhat close
// together (i.e. we only account for field5 and field6)
fn nearby_key_delta(start: &Key, end: &Key) -> u64 {
let start = (start.field5 as u64) << 32 | start.field6 as u64;
let end = (end.field5 as u64) << 32 | end.field6 as u64;
end - start
}
impl<'a> ShardedRange<'a> {
pub fn new(range: Range<Key>, shard_identity: &'a ShardIdentity) -> Self {
Self {
shard_identity,
range,
}
}
/// Break up this range into chunks, each of which has at least one local key in it if the
/// total range has at least one local key.
pub fn fragment(self, target_nblocks: u32) -> Vec<(u32, Range<Key>)> {
// Optimization for single-key case (e.g. logical size keys)
if self.range.end == self.range.start.add(1) {
return vec![(
if self.shard_identity.is_key_disposable(&self.range.start) {
0
} else {
1
},
self.range,
)];
}
if self.range.end.field1 != self.range.start.field1
|| self.range.end.field2 != self.range.start.field2
|| self.range.end.field3 != self.range.start.field3
|| self.range.end.field4 != self.range.start.field4
{
// Ranges that span relations are not fragmented. We only get these ranges as a result
// of operations that act on existing layers, so we trust that the existing range is
// reasonably small.
return vec![(u32::MAX, self.range)];
}
let mut fragments: Vec<(u32, Range<Key>)> = Vec::new();
let mut cursor = self.range.start;
while cursor < self.range.end {
let advance_by = self.advance_to_next_boundary(cursor);
let is_fragment_disposable = self.shard_identity.is_key_disposable(&cursor);
// If the previous fragment is undersized, then we seek to consume enough
// blocks to complete it.
let (want_blocks, merge_last_fragment) = match fragments.last_mut() {
Some(frag) if frag.0 < target_nblocks => (target_nblocks - frag.0, Some(frag)),
Some(frag) => {
// Prev block is complete, want the full number.
(
target_nblocks,
if is_fragment_disposable {
// If this current range will be empty (not shard-local data), we will merge into previous
Some(frag)
} else {
None
},
)
}
None => {
// First iteration, want the full number
(target_nblocks, None)
}
};
let advance_by = if is_fragment_disposable {
advance_by
} else {
std::cmp::min(advance_by, want_blocks)
};
let next_cursor = cursor.add(advance_by);
let this_frag = (
if is_fragment_disposable {
0
} else {
advance_by
},
cursor..next_cursor,
);
cursor = next_cursor;
if let Some(last_fragment) = merge_last_fragment {
// Previous fragment was short or this one is empty, merge into it
last_fragment.0 += this_frag.0;
last_fragment.1.end = this_frag.1.end;
} else {
fragments.push(this_frag);
}
}
fragments
}
/// Estimate the physical pages that are within this range, on this shard. This returns
/// u32::MAX if the range spans relations: this return value should be interpreted as "large".
pub fn page_count(&self) -> u32 {
let raw_size = Self::raw_size(&self.range);
if raw_size == u32::MAX {
return u32::MAX;
}
// Special case for single sharded tenants: our logical and physical sizes are the same
if self.shard_identity.count < ShardCount::new(2) {
return raw_size;
}
// Special cases for single keys like logical sizes
if self.range.end == self.range.start.add(1)
&& self.shard_identity.is_key_local(&self.range.start)
{
return 1;
}
// Normal path: step through stripes and part-stripes in the range, evaluate whether each one belongs
// to Self, and add the stripe's block count to our total if so.
let mut result: u64 = 0;
let mut cursor = self.range.start;
while cursor < self.range.end {
// Count up to the next stripe_size boundary or end of range
let advance_by = self.advance_to_next_boundary(cursor);
cursor = cursor.add(advance_by);
// If this blocks in this stripe belong to us, add them to our count
if !self.shard_identity.is_key_disposable(&cursor) {
result += advance_by as u64;
}
}
// Sharding should always decrease the number of pages we estimate, never increase it
debug_assert!(result <= raw_size as u64);
if result > u32::MAX as u64 {
u32::MAX
} else {
result as u32
}
}
/// Advance the cursor to the next potential fragment boundary: this is either
/// a stripe boundary, or the end of the range.
fn advance_to_next_boundary(&self, cursor: Key) -> u32 {
let distance_to_range_end = nearby_key_delta(&cursor, &self.range.end);
if self.shard_identity.count < ShardCount::new(2) {
// Optimization: don't bother stepping through stripes if the tenant isn't sharded.
return Self::raw_size(&self.range);
}
let stripe_index = cursor.field6 / self.shard_identity.stripe_size.0;
let stripe_remainder = self.shard_identity.stripe_size.0
- (cursor.field6 - stripe_index * self.shard_identity.stripe_size.0);
std::cmp::min(stripe_remainder as u64, distance_to_range_end) as u32
}
/// Whereas `page_count` estimates the number of pages physically in this range on this shard,
/// this function simply calculates the number of pages in the space, without accounting for those
/// pages that would not actually be stored on this node.
///
/// Don't use this function in code that works with physical entities like layer files.
fn raw_size(range: &Range<Key>) -> u32 {
let start = range.start;
let end = range.end;
if end.field1 != start.field1
|| end.field2 != start.field2
|| end.field3 != start.field3
|| end.field4 != start.field4
{
return u32::MAX;
}
// The check above ensures that keys only differ in low fields (i.e. are nearby)
let diff = nearby_key_delta(&start, &end);
if diff > u32::MAX as u64 {
u32::MAX
} else {
diff as u32
}
}
}
impl KeySpace {
///
/// Partition a key space into roughly chunks of roughly 'target_size' bytes
/// in each partition.
///
pub fn partition(&self, target_size: u64) -> KeyPartitioning {
pub fn partition(&self, shard_identity: &ShardIdentity, target_size: u64) -> KeyPartitioning {
// Assume that each value is 8k in size.
let target_nblocks = (target_size / BLCKSZ as u64) as usize;
let target_nblocks = (target_size / BLCKSZ as u64) as u32;
let mut parts = Vec::new();
let mut current_part = Vec::new();
let mut current_part_size: usize = 0;
for range in &self.ranges {
// If appending the next contiguous range in the keyspace to the current
// partition would cause it to be too large, start a new partition.
let this_size = key_range_size(range) as usize;
if current_part_size + this_size > target_nblocks && !current_part.is_empty() {
parts.push(KeySpace {
ranges: current_part,
});
current_part = Vec::new();
current_part_size = 0;
}
// While doing partitioning, wrap the range in ShardedRange so that our size calculations
// will respect shard striping rather than assuming all keys within a range are present.
let range = ShardedRange::new(range.clone(), shard_identity);
// If the next range is larger than 'target_size', split it into
// 'target_size' chunks.
let mut remain_size = this_size;
let mut start = range.start;
while remain_size > target_nblocks {
let next = start.add(target_nblocks as u32);
parts.push(KeySpace {
ranges: vec![start..next],
});
start = next;
remain_size -= target_nblocks
// Chunk up the range into parts that each contain up to target_size local blocks
for (range_size, range) in range.fragment(target_nblocks) {
// If appending the next contiguous range in the keyspace to the current
// partition would cause it to be too large, and our current partition
// covers at least one block that is physically present in this shard,
// then start a new partition
if current_part_size + range_size as usize > target_nblocks as usize
&& current_part_size > 0
{
parts.push(KeySpace {
ranges: current_part,
});
current_part = Vec::new();
current_part_size = 0;
}
current_part.push(range.start..range.end);
current_part_size += range_size as usize;
}
current_part.push(start..range.end);
current_part_size += remain_size;
}
// add last partition that wasn't full yet.
@@ -154,16 +359,16 @@ impl KeySpace {
self.ranges.last().map(|range| range.end)
}
#[allow(unused)]
pub fn total_size(&self) -> usize {
/// The size of the keyspace in pages, before accounting for sharding
pub fn total_raw_size(&self) -> usize {
self.ranges
.iter()
.map(|range| key_range_size(range) as usize)
.map(|range| ShardedRange::raw_size(range) as usize)
.sum()
}
pub fn is_empty(&self) -> bool {
self.total_size() == 0
self.total_raw_size() == 0
}
fn overlaps_at(&self, range: &Range<Key>) -> Option<usize> {
@@ -236,7 +441,7 @@ impl KeySpaceAccum {
#[inline(always)]
pub fn add_range(&mut self, range: Range<Key>) {
self.size += key_range_size(&range) as u64;
self.size += ShardedRange::raw_size(&range) as u64;
match self.accum.as_mut() {
Some(accum) => {
@@ -268,7 +473,9 @@ impl KeySpaceAccum {
std::mem::take(self).to_keyspace()
}
pub fn size(&self) -> u64 {
// The total number of keys in this object, ignoring any sharding effects that might cause some of
// the keys to be omitted in storage on this shard.
pub fn raw_size(&self) -> u64 {
self.size
}
}
@@ -324,36 +531,17 @@ impl KeySpaceRandomAccum {
}
}
#[inline(always)]
pub fn key_range_size(key_range: &Range<Key>) -> u32 {
let start = key_range.start;
let end = key_range.end;
if end.field1 != start.field1
|| end.field2 != start.field2
|| end.field3 != start.field3
|| end.field4 != start.field4
{
return u32::MAX;
}
let start = (start.field5 as u64) << 32 | start.field6 as u64;
let end = (end.field5 as u64) << 32 | end.field6 as u64;
let diff = end - start;
if diff > u32::MAX as u64 {
u32::MAX
} else {
diff as u32
}
}
pub fn singleton_range(key: Key) -> Range<Key> {
key..key.next()
}
#[cfg(test)]
mod tests {
use crate::{
models::ShardParameters,
shard::{ShardCount, ShardNumber},
};
use super::*;
use std::fmt::Write;
@@ -396,14 +584,17 @@ mod tests {
accum.add_range(range.clone());
}
let expected_size: u64 = ranges.iter().map(|r| key_range_size(r) as u64).sum();
assert_eq!(accum.size(), expected_size);
let expected_size: u64 = ranges
.iter()
.map(|r| ShardedRange::raw_size(r) as u64)
.sum();
assert_eq!(accum.raw_size(), expected_size);
assert_ks_eq(&accum.consume_keyspace(), ranges.clone());
assert_eq!(accum.size(), 0);
assert_eq!(accum.raw_size(), 0);
assert_ks_eq(&accum.consume_keyspace(), vec![]);
assert_eq!(accum.size(), 0);
assert_eq!(accum.raw_size(), 0);
for range in &ranges {
accum.add_range(range.clone());
@@ -700,4 +891,256 @@ mod tests {
]
);
}
#[test]
fn sharded_range_relation_gap() {
let shard_identity = ShardIdentity::new(
ShardNumber(0),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
let range = ShardedRange::new(
Range {
start: Key::from_hex("000000067F00000005000040100300000000").unwrap(),
end: Key::from_hex("000000067F00000005000040130000004000").unwrap(),
},
&shard_identity,
);
// Key range spans relations, expect MAX
assert_eq!(range.page_count(), u32::MAX);
}
#[test]
fn shard_identity_keyspaces_single_key() {
let shard_identity = ShardIdentity::new(
ShardNumber(1),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
let range = ShardedRange::new(
Range {
start: Key::from_hex("000000067f000000010000007000ffffffff").unwrap(),
end: Key::from_hex("000000067f00000001000000700100000000").unwrap(),
},
&shard_identity,
);
// Single-key range on logical size key
assert_eq!(range.page_count(), 1);
}
#[test]
fn shard_identity_keyspaces_forkno_gap() {
let shard_identity = ShardIdentity::new(
ShardNumber(1),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
let range = ShardedRange::new(
Range {
start: Key::from_hex("000000067f00000001000004df00fffffffe").unwrap(),
end: Key::from_hex("000000067f00000001000004df0100000003").unwrap(),
},
&shard_identity,
);
// Range spanning the end of one forkno and the start of the next, but not intersecting this shard's stripes
// This is technically an under-count, as the logical size key would be stored on this shard, but that's okay
// because page_count is allowed to under-count: it just mustn't over-count.
assert_eq!(range.page_count(), 0);
}
#[test]
fn shard_identity_keyspaces_one_relation() {
for shard_number in 0..4 {
let shard_identity = ShardIdentity::new(
ShardNumber(shard_number),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
let range = ShardedRange::new(
Range {
start: Key::from_hex("000000067f00000001000000ae0000000000").unwrap(),
end: Key::from_hex("000000067f00000001000000ae0000000001").unwrap(),
},
&shard_identity,
);
// Very simple case: range covering block zero of one relation, where that block maps to shard zero
if shard_number == 0 {
assert_eq!(range.page_count(), 1);
} else {
// Other shards should perceive the range's size as zero
assert_eq!(range.page_count(), 0);
}
}
}
/// Test helper: construct a ShardedRange and call fragment() on it, returning
/// the total page count in the range and the fragments.
fn do_fragment(
range_start: Key,
range_end: Key,
shard_identity: &ShardIdentity,
target_nblocks: u32,
) -> (u32, Vec<(u32, Range<Key>)>) {
let range = ShardedRange::new(
Range {
start: range_start,
end: range_end,
},
shard_identity,
);
let page_count = range.page_count();
let fragments = range.fragment(target_nblocks);
// Invariant: we always get at least one fragment
assert!(!fragments.is_empty());
if page_count > 0 {
// Invariant: every fragment must contain at least one shard-local page, if the
// total range contains at least one shard-local page
let all_nonzero = fragments.iter().all(|f| f.0 > 0);
if !all_nonzero {
eprintln!("Found a zero-length fragment: {:?}", fragments);
}
assert!(all_nonzero);
} else {
// A range with no shard-local pages should always be returned as a single fragment
assert_eq!(fragments, vec![(0, range_start..range_end)]);
}
(page_count, fragments)
}
/// Really simple tests for fragment(), on a range that just contains a single stripe
/// for a single tenant.
#[test]
fn sharded_range_fragment_simple() {
let shard_identity = ShardIdentity::new(
ShardNumber(0),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
// A range which we happen to know covers exactly one stripe which belongs to this shard
let input_start = Key::from_hex("000000067f00000001000000ae0000000000").unwrap();
let input_end = Key::from_hex("000000067f00000001000000ae0000008000").unwrap();
// Ask for stripe_size blocks, we get the whole stripe
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 32768),
(32768, vec![(32768, input_start..input_end)])
);
// Ask for more, we still get the whole stripe
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 10000000),
(32768, vec![(32768, input_start..input_end)])
);
// Ask for target_nblocks of half the stripe size, we get two halves
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 16384),
(
32768,
vec![
(16384, input_start..input_start.add(16384)),
(16384, input_start.add(16384)..input_end)
]
)
);
}
#[test]
fn sharded_range_fragment_multi_stripe() {
let shard_identity = ShardIdentity::new(
ShardNumber(0),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
// A range which covers multiple stripes, exactly one of which belongs to the current shard.
let input_start = Key::from_hex("000000067f00000001000000ae0000000000").unwrap();
let input_end = Key::from_hex("000000067f00000001000000ae0000020000").unwrap();
// Ask for all the blocks, get a fragment that covers the whole range but reports
// its size to be just the blocks belonging to our shard.
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 131072),
(32768, vec![(32768, input_start..input_end)])
);
// Ask for a sub-stripe quantity
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 16000),
(
32768,
vec![
(16000, input_start..input_start.add(16000)),
(16000, input_start.add(16000)..input_start.add(32000)),
(768, input_start.add(32000)..input_end),
]
)
);
// Try on a range that starts slightly after our owned stripe
assert_eq!(
do_fragment(input_start.add(1), input_end, &shard_identity, 131072),
(32767, vec![(32767, input_start.add(1)..input_end)])
);
}
/// Test that ShardedRange behaves properly when used on un-sharded data
#[test]
fn sharded_range_fragment_unsharded() {
let shard_identity = ShardIdentity::unsharded();
let input_start = Key::from_hex("000000067f00000001000000ae0000000000").unwrap();
let input_end = Key::from_hex("000000067f00000001000000ae0000010000").unwrap();
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 0x8000),
(
0x10000,
vec![
(0x8000, input_start..input_start.add(0x8000)),
(0x8000, input_start.add(0x8000)..input_start.add(0x10000))
]
)
);
}
#[test]
fn sharded_range_fragment_cross_relation() {
let shard_identity = ShardIdentity::unsharded();
// A range that spans relations: expect fragmentation to give up and return a u32::MAX size
let input_start = Key::from_hex("000000067f00000001000000ae0000000000").unwrap();
let input_end = Key::from_hex("000000068f00000001000000ae0000010000").unwrap();
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 0x8000),
(u32::MAX, vec![(u32::MAX, input_start..input_end),])
);
// Same, but using a sharded identity
let shard_identity = ShardIdentity::new(
ShardNumber(0),
ShardCount::new(4),
ShardParameters::DEFAULT_STRIPE_SIZE,
)
.unwrap();
assert_eq!(
do_fragment(input_start, input_end, &shard_identity, 0x8000),
(u32::MAX, vec![(u32::MAX, input_start..input_end),])
);
}
}

View File

@@ -451,7 +451,7 @@ impl ShardIdentity {
/// An identity with number=0 count=0 is a "none" identity, which represents legacy
/// tenants. Modern single-shard tenants should not use this: they should
/// have number=0 count=1.
pub fn unsharded() -> Self {
pub const fn unsharded() -> Self {
Self {
number: ShardNumber(0),
count: ShardCount(0),

View File

@@ -18,6 +18,7 @@
//! database size. For example, if the logical database size is 10 GB, we would
//! generate new image layers every 10 GB of WAL.
use futures::StreamExt;
use pageserver_api::shard::ShardIdentity;
use tracing::{debug, info};
use std::collections::{HashSet, VecDeque};
@@ -125,6 +126,7 @@ async fn compact_level<E: CompactionJobExecutor>(
}
let mut state = LevelCompactionState {
shard_identity: *executor.get_shard_identity(),
target_file_size,
_lsn_range: lsn_range.clone(),
layers: layer_fragments,
@@ -164,6 +166,8 @@ struct LevelCompactionState<'a, E>
where
E: CompactionJobExecutor,
{
shard_identity: ShardIdentity,
// parameters
target_file_size: u64,
@@ -366,6 +370,7 @@ where
.executor
.get_keyspace(&job.key_range, job.lsn_range.end, ctx)
.await?,
&self.shard_identity,
) * 8192;
let wal_size = job
@@ -430,7 +435,7 @@ where
keyspace,
self.target_file_size / 8192,
);
while let Some(key_range) = window.choose_next_image() {
while let Some(key_range) = window.choose_next_image(&self.shard_identity) {
new_jobs.push(CompactionJob::<E> {
key_range,
lsn_range: job.lsn_range.clone(),
@@ -623,7 +628,12 @@ impl<K: CompactionKey> KeyspaceWindowPos<K> {
}
// Advance the cursor until it reaches 'target_keysize'.
fn advance_until_size(&mut self, w: &KeyspaceWindowHead<K>, max_size: u64) {
fn advance_until_size(
&mut self,
w: &KeyspaceWindowHead<K>,
max_size: u64,
shard_identity: &ShardIdentity,
) {
while self.accum_keysize < max_size && !self.reached_end(w) {
let curr_range = &w.keyspace[self.keyspace_idx];
if self.end_key < curr_range.start {
@@ -632,7 +642,7 @@ impl<K: CompactionKey> KeyspaceWindowPos<K> {
}
// We're now within 'curr_range'. Can we advance past it completely?
let distance = K::key_range_size(&(self.end_key..curr_range.end));
let distance = K::key_range_size(&(self.end_key..curr_range.end), shard_identity);
if (self.accum_keysize + distance as u64) < max_size {
// oh yeah, it fits
self.end_key = curr_range.end;
@@ -641,7 +651,7 @@ impl<K: CompactionKey> KeyspaceWindowPos<K> {
} else {
// advance within the range
let skip_key = self.end_key.skip_some();
let distance = K::key_range_size(&(self.end_key..skip_key));
let distance = K::key_range_size(&(self.end_key..skip_key), shard_identity);
if (self.accum_keysize + distance as u64) < max_size {
self.end_key = skip_key;
self.accum_keysize += distance as u64;
@@ -677,7 +687,7 @@ where
}
}
fn choose_next_image(&mut self) -> Option<Range<K>> {
fn choose_next_image(&mut self, shard_identity: &ShardIdentity) -> Option<Range<K>> {
if self.start_pos.keyspace_idx == self.head.keyspace.len() {
// we've reached the end
return None;
@@ -687,6 +697,7 @@ where
next_pos.advance_until_size(
&self.head,
self.start_pos.accum_keysize + self.head.target_keysize,
shard_identity,
);
// See if we can gobble up the rest of the keyspace if we stretch out the layer, up to
@@ -695,6 +706,7 @@ where
end_pos.advance_until_size(
&self.head,
self.start_pos.accum_keysize + (self.head.target_keysize * 5 / 4),
shard_identity,
);
if end_pos.reached_end(&self.head) {
// gobble up any unused keyspace between the last used key and end of the range

View File

@@ -5,6 +5,7 @@ use crate::interface::*;
use futures::future::BoxFuture;
use futures::{Stream, StreamExt};
use itertools::Itertools;
use pageserver_api::shard::ShardIdentity;
use pin_project_lite::pin_project;
use std::collections::BinaryHeap;
use std::collections::VecDeque;
@@ -13,11 +14,17 @@ use std::ops::{DerefMut, Range};
use std::pin::Pin;
use std::task::{ready, Poll};
pub fn keyspace_total_size<K>(keyspace: &CompactionKeySpace<K>) -> u64
pub fn keyspace_total_size<K>(
keyspace: &CompactionKeySpace<K>,
shard_identity: &ShardIdentity,
) -> u64
where
K: CompactionKey,
{
keyspace.iter().map(|r| K::key_range_size(r) as u64).sum()
keyspace
.iter()
.map(|r| K::key_range_size(r, shard_identity) as u64)
.sum()
}
pub fn overlaps_with<T: Ord>(a: &Range<T>, b: &Range<T>) -> bool {

View File

@@ -4,7 +4,7 @@
//! All the heavy lifting is done by the create_image and create_delta
//! functions that the implementor provides.
use futures::Future;
use pageserver_api::{key::Key, keyspace::key_range_size};
use pageserver_api::{key::Key, keyspace::ShardedRange, shard::ShardIdentity};
use std::ops::Range;
use utils::lsn::Lsn;
@@ -32,6 +32,8 @@ pub trait CompactionJobExecutor {
// Functions that the planner uses to support its decisions
// ----
fn get_shard_identity(&self) -> &ShardIdentity;
/// Return all layers that overlap the given bounding box.
fn get_layers(
&mut self,
@@ -98,7 +100,7 @@ pub trait CompactionKey: std::cmp::Ord + Clone + Copy + std::fmt::Display {
///
/// This returns u32, for compatibility with Repository::key. If the
/// distance is larger, return u32::MAX.
fn key_range_size(key_range: &Range<Self>) -> u32;
fn key_range_size(key_range: &Range<Self>, shard_identity: &ShardIdentity) -> u32;
// return "self + 1"
fn next(&self) -> Self;
@@ -113,8 +115,8 @@ impl CompactionKey for Key {
const MIN: Self = Self::MIN;
const MAX: Self = Self::MAX;
fn key_range_size(r: &std::ops::Range<Self>) -> u32 {
key_range_size(r)
fn key_range_size(r: &std::ops::Range<Self>, shard_identity: &ShardIdentity) -> u32 {
ShardedRange::new(r.clone(), shard_identity).page_count()
}
fn next(&self) -> Key {
(self as &Key).next()

View File

@@ -3,6 +3,7 @@ mod draw;
use draw::{LayerTraceEvent, LayerTraceFile, LayerTraceOp};
use futures::StreamExt;
use pageserver_api::shard::ShardIdentity;
use rand::Rng;
use tracing::info;
@@ -71,7 +72,7 @@ impl interface::CompactionKey for Key {
const MIN: Self = u64::MIN;
const MAX: Self = u64::MAX;
fn key_range_size(key_range: &Range<Self>) -> u32 {
fn key_range_size(key_range: &Range<Self>, _shard_identity: &ShardIdentity) -> u32 {
std::cmp::min(key_range.end - key_range.start, u32::MAX as u64) as u32
}
@@ -434,6 +435,11 @@ impl interface::CompactionJobExecutor for MockTimeline {
type ImageLayer = Arc<MockImageLayer>;
type RequestContext = MockRequestContext;
fn get_shard_identity(&self) -> &ShardIdentity {
static IDENTITY: ShardIdentity = ShardIdentity::unsharded();
&IDENTITY
}
async fn get_layers(
&mut self,
key_range: &Range<Self::Key>,

View File

@@ -263,7 +263,10 @@ where
.timeline
.get_slru_keyspace(Version::Lsn(self.lsn), self.ctx)
.await?
.partition(Timeline::MAX_GET_VECTORED_KEYS * BLCKSZ as u64);
.partition(
self.timeline.get_shard_identity(),
Timeline::MAX_GET_VECTORED_KEYS * BLCKSZ as u64,
);
let mut slru_builder = SlruSegmentsBuilder::new(&mut self.ar);

View File

@@ -406,7 +406,7 @@ impl InMemoryLayer {
}
}
let keyspace_size = keyspace.total_size();
let keyspace_size = keyspace.total_raw_size();
let mut completed_keys = HashSet::new();
while completed_keys.len() < keyspace_size && !planned_block_reads.is_empty() {

View File

@@ -936,7 +936,7 @@ impl Timeline {
return Err(GetVectoredError::InvalidLsn(lsn));
}
let key_count = keyspace.total_size().try_into().unwrap();
let key_count = keyspace.total_raw_size().try_into().unwrap();
if key_count > Timeline::MAX_GET_VECTORED_KEYS {
return Err(GetVectoredError::Oversized(key_count));
}
@@ -1076,7 +1076,7 @@ impl Timeline {
mut reconstruct_state: ValuesReconstructState,
ctx: &RequestContext,
) -> Result<BTreeMap<Key, Result<Bytes, PageReconstructError>>, GetVectoredError> {
let get_kind = if keyspace.total_size() == 1 {
let get_kind = if keyspace.total_raw_size() == 1 {
GetKind::Singular
} else {
GetKind::Vectored
@@ -3193,7 +3193,7 @@ impl Timeline {
}
}
if keyspace.total_size() == 0 || timeline.ancestor_timeline.is_none() {
if keyspace.total_raw_size() == 0 || timeline.ancestor_timeline.is_none() {
break;
}
@@ -3206,7 +3206,7 @@ impl Timeline {
timeline = &*timeline_owned;
}
if keyspace.total_size() != 0 {
if keyspace.total_raw_size() != 0 {
return Err(GetVectoredError::MissingKey(keyspace.start().unwrap()));
}
@@ -3897,7 +3897,7 @@ impl Timeline {
}
let keyspace = self.collect_keyspace(lsn, ctx).await?;
let partitioning = keyspace.partition(partition_size);
let partitioning = keyspace.partition(&self.shard_identity, partition_size);
*partitioning_guard = (partitioning, lsn);
@@ -4040,7 +4040,7 @@ impl Timeline {
key = key.next();
// Maybe flush `key_rest_accum`
if key_request_accum.size() >= Timeline::MAX_GET_VECTORED_KEYS
if key_request_accum.raw_size() >= Timeline::MAX_GET_VECTORED_KEYS
|| last_key_in_range
{
let results = self

View File

@@ -15,7 +15,7 @@ use anyhow::{anyhow, Context};
use enumset::EnumSet;
use fail::fail_point;
use itertools::Itertools;
use pageserver_api::shard::TenantShardId;
use pageserver_api::shard::{ShardIdentity, TenantShardId};
use tokio_util::sync::CancellationToken;
use tracing::{debug, info, info_span, trace, warn, Instrument};
use utils::id::TimelineId;
@@ -831,6 +831,10 @@ impl CompactionJobExecutor for TimelineAdaptor {
type RequestContext = crate::context::RequestContext;
fn get_shard_identity(&self) -> &ShardIdentity {
self.timeline.get_shard_identity()
}
async fn get_layers(
&mut self,
key_range: &Range<Key>,

View File

@@ -1,3 +1,4 @@
import json
import os
import time
from collections import defaultdict
@@ -1243,3 +1244,93 @@ def test_sharding_unlogged_relation(neon_env_builder: NeonEnvBuilder):
# Ensure that post-endpoint-restart modifications are ingested happily by pageserver
wait_for_last_flush_lsn(env, ep, tenant_id, timeline_id)
# Stripe sizes in number of pages.
TINY_STRIPES = 16
LARGE_STRIPES = 32768
@pytest.mark.parametrize("stripe_size", [TINY_STRIPES, LARGE_STRIPES])
def test_sharding_compaction(neon_env_builder: NeonEnvBuilder, stripe_size: int):
"""
Use small stripes, small layers, and small compaction thresholds to exercise how compaction
and image layer generation interacts with sharding.
"""
compaction_target_size = 128 * 1024
TENANT_CONF = {
# small checkpointing and compaction targets to ensure we generate many upload operations
"checkpoint_distance": f"{128 * 1024}",
"compaction_threshold": "1",
"compaction_target_size": f"{compaction_target_size}",
# no PITR horizon, we specify the horizon when we request on-demand GC
"pitr_interval": "0s",
# disable background compaction and GC. We invoke it manually when we want it to happen.
"gc_period": "0s",
"compaction_period": "0s",
# create image layers eagerly: we want to exercise image layer creation in this test.
"image_creation_threshold": "1",
"image_layer_creation_check_threshold": 0,
}
neon_env_builder.num_pageservers = 4
env = neon_env_builder.init_start(
initial_tenant_conf=TENANT_CONF,
initial_tenant_shard_count=4,
initial_tenant_shard_stripe_size=stripe_size,
)
tenant_id = env.initial_tenant
timeline_id = env.initial_timeline
workload = Workload(env, tenant_id, timeline_id)
workload.init()
workload.write_rows(64)
for _i in range(0, 10):
# Each of these does some writes then a checkpoint: because we set image_creation_threshold to 1,
# these should result in image layers each time we write some data into a shard, and also shards
# recieving less data hitting their "empty image layer" path (wherre they should skip writing the layer,
# rather than asserting)
workload.churn_rows(64)
# Assert that we got some image layers: this is important because this test's purpose is to exercise the sharding changes
# to Timeline::create_image_layers, so if we weren't creating any image layers we wouldn't be doing our job.
shard_has_image_layers = []
for shard in env.storage_controller.locate(tenant_id):
pageserver = env.get_pageserver(shard["node_id"])
shard_id = shard["shard_id"]
layer_map = pageserver.http_client().layer_map_info(shard_id, timeline_id)
image_layer_sizes = {}
for layer in layer_map.historic_layers:
if layer.kind == "Image":
image_layer_sizes[layer.layer_file_name] = layer.layer_file_size
# Pageserver should assert rather than emit an empty layer file, but double check here
assert layer.layer_file_size is not None
assert layer.layer_file_size > 0
shard_has_image_layers.append(len(image_layer_sizes) > 1)
log.info(f"Shard {shard_id} image layer sizes: {json.dumps(image_layer_sizes, indent=2)}")
if stripe_size == TINY_STRIPES:
# Checking the average size validates that our keyspace partitioning is properly respecting sharding: if
# it was not, we would tend to get undersized layers because the partitioning would overestimate the physical
# data in a keyrange.
#
# We only do this check with tiny stripes, because large stripes may not give all shards enough
# data to have statistically significant image layers
avg_size = sum(v for v in image_layer_sizes.values()) / len(image_layer_sizes) # type: ignore
log.info(f"Shard {shard_id} average image layer size: {avg_size}")
assert avg_size > compaction_target_size / 2
if stripe_size == TINY_STRIPES:
# Expect writes were scattered across all pageservers: they should all have compacted some image layers
assert all(shard_has_image_layers)
else:
# With large stripes, it is expected that most of our writes went to one pageserver, so we just require
# that at least one of them has some image layers.
assert any(shard_has_image_layers)
# Assert that everything is still readable
workload.validate()