mirror of
https://github.com/neondatabase/neon.git
synced 2026-02-13 15:40:37 +00:00
Compare commits
20 Commits
refactor-c
...
jcsp/inges
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
5664eadb17 | ||
|
|
1a99aa4834 | ||
|
|
35df89dcfe | ||
|
|
28280683ab | ||
|
|
eed100b21e | ||
|
|
4bd26c54e5 | ||
|
|
e461a711d0 | ||
|
|
93dacd75aa | ||
|
|
06428e856e | ||
|
|
fce68fe84e | ||
|
|
fd60904376 | ||
|
|
bf3e767b35 | ||
|
|
513cafd72b | ||
|
|
c2d5395a00 | ||
|
|
d152a57c29 | ||
|
|
a8be0f3376 | ||
|
|
5dcfe1c4b8 | ||
|
|
ae7d635098 | ||
|
|
137cbb4db4 | ||
|
|
74eda0b0b7 |
1
Cargo.lock
generated
1
Cargo.lock
generated
@@ -6762,6 +6762,7 @@ dependencies = [
|
||||
"serde_path_to_error",
|
||||
"serde_with",
|
||||
"signal-hook",
|
||||
"smallvec",
|
||||
"strum",
|
||||
"strum_macros",
|
||||
"thiserror",
|
||||
|
||||
@@ -35,6 +35,7 @@ routerify.workspace = true
|
||||
serde.workspace = true
|
||||
serde_json.workspace = true
|
||||
signal-hook.workspace = true
|
||||
smallvec.workspace = true
|
||||
thiserror.workspace = true
|
||||
tokio.workspace = true
|
||||
tokio-tar.workspace = true
|
||||
|
||||
@@ -1,11 +1,15 @@
|
||||
use std::{alloc::Layout, cmp::Ordering, ops::RangeBounds};
|
||||
|
||||
use smallvec::SmallVec;
|
||||
|
||||
#[derive(Clone, Copy, Debug, PartialEq, Eq)]
|
||||
pub enum VecMapOrdering {
|
||||
Greater,
|
||||
GreaterOrEqual,
|
||||
}
|
||||
|
||||
const INLINE_ELEMENTS: usize = 1;
|
||||
|
||||
/// Ordered map datastructure implemented in a Vec.
|
||||
/// Append only - can only add keys that are larger than the
|
||||
/// current max key.
|
||||
@@ -13,7 +17,7 @@ pub enum VecMapOrdering {
|
||||
/// during `VecMap` construction.
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct VecMap<K, V> {
|
||||
data: Vec<(K, V)>,
|
||||
data: SmallVec<[(K, V); INLINE_ELEMENTS]>,
|
||||
ordering: VecMapOrdering,
|
||||
}
|
||||
|
||||
@@ -37,14 +41,14 @@ pub enum VecMapError {
|
||||
impl<K: Ord, V> VecMap<K, V> {
|
||||
pub fn new(ordering: VecMapOrdering) -> Self {
|
||||
Self {
|
||||
data: Vec::new(),
|
||||
data: Default::default(),
|
||||
ordering,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn with_capacity(capacity: usize, ordering: VecMapOrdering) -> Self {
|
||||
Self {
|
||||
data: Vec::with_capacity(capacity),
|
||||
data: SmallVec::with_capacity(capacity),
|
||||
ordering,
|
||||
}
|
||||
}
|
||||
@@ -95,6 +99,10 @@ impl<K: Ord, V> VecMap<K, V> {
|
||||
Ok(delta_size)
|
||||
}
|
||||
|
||||
pub fn append_fast(&mut self, key: K, value: V) {
|
||||
self.data.push((key, value))
|
||||
}
|
||||
|
||||
/// Update the maximum key value pair or add a new key value pair to the map.
|
||||
/// If `key` is not respective of the `self` ordering no updates or additions
|
||||
/// will occur and `InvalidKey` error will be returned.
|
||||
@@ -135,11 +143,11 @@ impl<K: Ord, V> VecMap<K, V> {
|
||||
|
||||
(
|
||||
VecMap {
|
||||
data: self.data[..split_idx].to_vec(),
|
||||
data: SmallVec::from(&self.data[..split_idx]),
|
||||
ordering: self.ordering,
|
||||
},
|
||||
VecMap {
|
||||
data: self.data[split_idx..].to_vec(),
|
||||
data: SmallVec::from(&self.data[split_idx..]),
|
||||
ordering: self.ordering,
|
||||
},
|
||||
)
|
||||
@@ -186,7 +194,10 @@ impl<K: Ord, V> VecMap<K, V> {
|
||||
/// Instrument an operation on the underlying [`Vec`].
|
||||
/// Will panic if the operation decreases capacity.
|
||||
/// Returns the increase in memory usage caused by the op.
|
||||
fn instrument_vec_op(&mut self, op: impl FnOnce(&mut Vec<(K, V)>)) -> usize {
|
||||
fn instrument_vec_op(
|
||||
&mut self,
|
||||
op: impl FnOnce(&mut SmallVec<[(K, V); INLINE_ELEMENTS]>),
|
||||
) -> usize {
|
||||
let old_cap = self.data.capacity();
|
||||
op(&mut self.data);
|
||||
let new_cap = self.data.capacity();
|
||||
@@ -226,7 +237,7 @@ impl<K: Ord, V> VecMap<K, V> {
|
||||
|
||||
impl<K: Ord, V> IntoIterator for VecMap<K, V> {
|
||||
type Item = (K, V);
|
||||
type IntoIter = std::vec::IntoIter<(K, V)>;
|
||||
type IntoIter = smallvec::IntoIter<[(K, V); INLINE_ELEMENTS]>;
|
||||
|
||||
fn into_iter(self) -> Self::IntoIter {
|
||||
self.data.into_iter()
|
||||
|
||||
@@ -108,3 +108,7 @@ harness = false
|
||||
[[bench]]
|
||||
name = "bench_walredo"
|
||||
harness = false
|
||||
|
||||
[[bench]]
|
||||
name = "bench_ingest"
|
||||
harness = false
|
||||
|
||||
250
pageserver/benches/bench_ingest.rs
Normal file
250
pageserver/benches/bench_ingest.rs
Normal file
@@ -0,0 +1,250 @@
|
||||
use std::{env, num::NonZeroUsize};
|
||||
|
||||
use bytes::Bytes;
|
||||
use camino::Utf8PathBuf;
|
||||
use criterion::{criterion_group, criterion_main, Criterion};
|
||||
use pageserver::{
|
||||
config::PageServerConf,
|
||||
context::{DownloadBehavior, RequestContext},
|
||||
l0_flush::{L0FlushConfig, L0FlushGlobalState},
|
||||
page_cache,
|
||||
repository::Value,
|
||||
task_mgr::TaskKind,
|
||||
tenant::storage_layer::{InMemoryLayer, SerializedBatch},
|
||||
virtual_file::{self, api::IoEngineKind},
|
||||
};
|
||||
use pageserver_api::{key::Key, shard::TenantShardId};
|
||||
use utils::{
|
||||
bin_ser::BeSer,
|
||||
id::{TenantId, TimelineId},
|
||||
};
|
||||
|
||||
// A very cheap hash for generating non-sequential keys.
|
||||
fn murmurhash32(mut h: u32) -> u32 {
|
||||
h ^= h >> 16;
|
||||
h = h.wrapping_mul(0x85ebca6b);
|
||||
h ^= h >> 13;
|
||||
h = h.wrapping_mul(0xc2b2ae35);
|
||||
h ^= h >> 16;
|
||||
h
|
||||
}
|
||||
|
||||
enum KeyLayout {
|
||||
/// Sequential unique keys
|
||||
Sequential,
|
||||
/// Random unique keys
|
||||
Random,
|
||||
/// Random keys, but only use the bits from the mask of them
|
||||
RandomReuse(u32),
|
||||
}
|
||||
|
||||
enum WriteDelta {
|
||||
Yes,
|
||||
No,
|
||||
}
|
||||
|
||||
async fn ingest(
|
||||
conf: &'static PageServerConf,
|
||||
put_size: usize,
|
||||
put_count: usize,
|
||||
key_layout: KeyLayout,
|
||||
write_delta: WriteDelta,
|
||||
) -> anyhow::Result<()> {
|
||||
let mut lsn = utils::lsn::Lsn(1000);
|
||||
let mut key = Key::from_i128(0x0);
|
||||
|
||||
let timeline_id = TimelineId::generate();
|
||||
let tenant_id = TenantId::generate();
|
||||
let tenant_shard_id = TenantShardId::unsharded(tenant_id);
|
||||
|
||||
tokio::fs::create_dir_all(conf.timeline_path(&tenant_shard_id, &timeline_id)).await?;
|
||||
|
||||
let ctx = RequestContext::new(TaskKind::DebugTool, DownloadBehavior::Error);
|
||||
|
||||
let layer = InMemoryLayer::create(conf, timeline_id, tenant_shard_id, lsn, &ctx).await?;
|
||||
|
||||
let value = Value::Image(Bytes::from(vec![0u8; put_size]));
|
||||
let ctx = RequestContext::new(
|
||||
pageserver::task_mgr::TaskKind::WalReceiverConnectionHandler,
|
||||
pageserver::context::DownloadBehavior::Download,
|
||||
);
|
||||
|
||||
let batch_pages = 10000;
|
||||
let mut batch_values = vec![];
|
||||
|
||||
for i in 0..put_count {
|
||||
lsn += put_size as u64;
|
||||
|
||||
// Generate lots of keys within a single relation, which simulates the typical bulk ingest case: people
|
||||
// usually care the most about write performance when they're blasting a huge batch of data into a huge table.
|
||||
match key_layout {
|
||||
KeyLayout::Sequential => {
|
||||
// Use sequential order to illustrate the experience a user is likely to have
|
||||
// when ingesting bulk data.
|
||||
key.field6 = i as u32;
|
||||
}
|
||||
KeyLayout::Random => {
|
||||
// Use random-order keys to avoid giving a false advantage to data structures that are
|
||||
// faster when inserting on the end.
|
||||
key.field6 = murmurhash32(i as u32);
|
||||
}
|
||||
KeyLayout::RandomReuse(mask) => {
|
||||
// Use low bits only, to limit cardinality
|
||||
key.field6 = murmurhash32(i as u32) & mask;
|
||||
}
|
||||
}
|
||||
|
||||
batch_values.push((key, lsn, value.clone()));
|
||||
|
||||
if batch_values.len() >= batch_pages {
|
||||
let write_batch = std::mem::take(&mut batch_values);
|
||||
let batch = SerializedBatch::from_values(write_batch);
|
||||
|
||||
layer.put_batch(&batch, &ctx).await?;
|
||||
}
|
||||
}
|
||||
if !batch_values.is_empty() {
|
||||
let batch = SerializedBatch::from_values(vec![(key, lsn, value.clone())]);
|
||||
|
||||
layer.put_batch(&batch, &ctx).await?;
|
||||
}
|
||||
layer.freeze(lsn + 1).await;
|
||||
|
||||
if matches!(write_delta, WriteDelta::Yes) {
|
||||
let l0_flush_state = L0FlushGlobalState::new(L0FlushConfig::Direct {
|
||||
max_concurrency: NonZeroUsize::new(1).unwrap(),
|
||||
});
|
||||
let (_desc, path) = layer
|
||||
.write_to_disk(&ctx, None, l0_flush_state.inner())
|
||||
.await?
|
||||
.unwrap();
|
||||
tokio::fs::remove_file(path).await?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Wrapper to instantiate a tokio runtime
|
||||
fn ingest_main(
|
||||
conf: &'static PageServerConf,
|
||||
put_size: usize,
|
||||
put_count: usize,
|
||||
key_layout: KeyLayout,
|
||||
write_delta: WriteDelta,
|
||||
) {
|
||||
let runtime = tokio::runtime::Builder::new_current_thread()
|
||||
.enable_all()
|
||||
.build()
|
||||
.unwrap();
|
||||
|
||||
runtime.block_on(async move {
|
||||
let r = ingest(conf, put_size, put_count, key_layout, write_delta).await;
|
||||
if let Err(e) = r {
|
||||
panic!("{e:?}");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/// Declare a series of benchmarks for the Pageserver's ingest write path.
|
||||
///
|
||||
/// This benchmark does not include WAL decode: it starts at InMemoryLayer::put_value, and ends either
|
||||
/// at freezing the ephemeral layer, or writing the ephemeral layer out to an L0 (depending on whether WriteDelta is set).
|
||||
///
|
||||
/// Genuine disk I/O is used, so expect results to differ depending on storage. However, when running on
|
||||
/// a fast disk, CPU is the bottleneck at time of writing.
|
||||
fn criterion_benchmark(c: &mut Criterion) {
|
||||
let temp_dir_parent: Utf8PathBuf = env::current_dir().unwrap().try_into().unwrap();
|
||||
let temp_dir = camino_tempfile::tempdir_in(temp_dir_parent).unwrap();
|
||||
eprintln!("Data directory: {}", temp_dir.path());
|
||||
|
||||
let conf: &'static PageServerConf = Box::leak(Box::new(
|
||||
pageserver::config::PageServerConf::dummy_conf(temp_dir.path().to_path_buf()),
|
||||
));
|
||||
virtual_file::init(16384, IoEngineKind::TokioEpollUring);
|
||||
page_cache::init(conf.page_cache_size);
|
||||
|
||||
{
|
||||
let mut group = c.benchmark_group("ingest-small-values");
|
||||
let put_size = 100usize;
|
||||
let put_count = 128 * 1024 * 1024 / put_size;
|
||||
group.throughput(criterion::Throughput::Bytes((put_size * put_count) as u64));
|
||||
group.sample_size(10);
|
||||
group.bench_function("ingest 128MB/100b seq", |b| {
|
||||
b.iter(|| {
|
||||
ingest_main(
|
||||
conf,
|
||||
put_size,
|
||||
put_count,
|
||||
KeyLayout::Sequential,
|
||||
WriteDelta::Yes,
|
||||
)
|
||||
})
|
||||
});
|
||||
group.bench_function("ingest 128MB/100b rand", |b| {
|
||||
b.iter(|| {
|
||||
ingest_main(
|
||||
conf,
|
||||
put_size,
|
||||
put_count,
|
||||
KeyLayout::Random,
|
||||
WriteDelta::Yes,
|
||||
)
|
||||
})
|
||||
});
|
||||
group.bench_function("ingest 128MB/100b rand-1024keys", |b| {
|
||||
b.iter(|| {
|
||||
ingest_main(
|
||||
conf,
|
||||
put_size,
|
||||
put_count,
|
||||
KeyLayout::RandomReuse(0x3ff),
|
||||
WriteDelta::Yes,
|
||||
)
|
||||
})
|
||||
});
|
||||
group.bench_function("ingest 128MB/100b seq, no delta", |b| {
|
||||
b.iter(|| {
|
||||
ingest_main(
|
||||
conf,
|
||||
put_size,
|
||||
put_count,
|
||||
KeyLayout::Sequential,
|
||||
WriteDelta::No,
|
||||
)
|
||||
})
|
||||
});
|
||||
}
|
||||
|
||||
{
|
||||
let mut group = c.benchmark_group("ingest-big-values");
|
||||
let put_size = 8192usize;
|
||||
let put_count = 128 * 1024 * 1024 / put_size;
|
||||
group.throughput(criterion::Throughput::Bytes((put_size * put_count) as u64));
|
||||
group.sample_size(10);
|
||||
group.bench_function("ingest 128MB/8k seq", |b| {
|
||||
b.iter(|| {
|
||||
ingest_main(
|
||||
conf,
|
||||
put_size,
|
||||
put_count,
|
||||
KeyLayout::Sequential,
|
||||
WriteDelta::Yes,
|
||||
)
|
||||
})
|
||||
});
|
||||
group.bench_function("ingest 128MB/8k seq, no delta", |b| {
|
||||
b.iter(|| {
|
||||
ingest_main(
|
||||
conf,
|
||||
put_size,
|
||||
put_count,
|
||||
KeyLayout::Sequential,
|
||||
WriteDelta::No,
|
||||
)
|
||||
})
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
criterion_group!(benches, criterion_benchmark);
|
||||
criterion_main!(benches);
|
||||
@@ -24,7 +24,7 @@ impl Default for L0FlushConfig {
|
||||
#[derive(Clone)]
|
||||
pub struct L0FlushGlobalState(Arc<Inner>);
|
||||
|
||||
pub(crate) enum Inner {
|
||||
pub enum Inner {
|
||||
PageCached,
|
||||
Direct { semaphore: tokio::sync::Semaphore },
|
||||
}
|
||||
@@ -40,7 +40,7 @@ impl L0FlushGlobalState {
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn inner(&self) -> &Arc<Inner> {
|
||||
pub fn inner(&self) -> &Arc<Inner> {
|
||||
&self.0
|
||||
}
|
||||
}
|
||||
|
||||
@@ -15,7 +15,6 @@ use crate::{aux_file, repository::*};
|
||||
use anyhow::{ensure, Context};
|
||||
use bytes::{Buf, Bytes, BytesMut};
|
||||
use enum_map::Enum;
|
||||
use itertools::Itertools;
|
||||
use pageserver_api::key::{
|
||||
dbdir_key_range, rel_block_to_key, rel_dir_to_key, rel_key_range, rel_size_to_key,
|
||||
relmap_file_key, repl_origin_key, repl_origin_key_range, slru_block_to_key, slru_dir_to_key,
|
||||
@@ -37,7 +36,6 @@ use tokio_util::sync::CancellationToken;
|
||||
use tracing::{debug, info, trace, warn};
|
||||
use utils::bin_ser::DeserializeError;
|
||||
use utils::pausable_failpoint;
|
||||
use utils::vec_map::{VecMap, VecMapOrdering};
|
||||
use utils::{bin_ser::BeSer, lsn::Lsn};
|
||||
|
||||
/// 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.
|
||||
@@ -174,6 +172,7 @@ impl Timeline {
|
||||
pending_deletions: Vec::new(),
|
||||
pending_nblocks: 0,
|
||||
pending_directory_entries: Vec::new(),
|
||||
pending_bytes: 0,
|
||||
lsn,
|
||||
}
|
||||
}
|
||||
@@ -1058,14 +1057,26 @@ pub struct DatadirModification<'a> {
|
||||
/// For special "directory" keys that store key-value maps, track the size of the map
|
||||
/// if it was updated in this modification.
|
||||
pending_directory_entries: Vec<(DirectoryKind, usize)>,
|
||||
|
||||
/// An **approximation** of how large our EphemeralFile write will be when committed.
|
||||
pending_bytes: usize,
|
||||
}
|
||||
|
||||
impl<'a> DatadirModification<'a> {
|
||||
// When a DatadirModification is committed, we do a monolithic serialization of all its contents. WAL records can
|
||||
// contain multiple pages, so the pageserver's record-based batch size isn't sufficient to bound this allocation: we
|
||||
// additionally specify a limit on how much payload a DatadirModification may contain before it should be committed.
|
||||
pub(crate) const MAX_PENDING_BYTES: usize = 8 * 1024 * 1024;
|
||||
|
||||
/// Get the current lsn
|
||||
pub(crate) fn get_lsn(&self) -> Lsn {
|
||||
self.lsn
|
||||
}
|
||||
|
||||
pub(crate) fn approx_pending_bytes(&self) -> usize {
|
||||
self.pending_bytes
|
||||
}
|
||||
|
||||
/// Set the current lsn
|
||||
pub(crate) fn set_lsn(&mut self, lsn: Lsn) -> anyhow::Result<()> {
|
||||
ensure!(
|
||||
@@ -1793,11 +1804,12 @@ impl<'a> DatadirModification<'a> {
|
||||
// Flush relation and SLRU data blocks, keep metadata.
|
||||
let mut retained_pending_updates = HashMap::<_, Vec<_>>::new();
|
||||
for (key, values) in self.pending_updates.drain() {
|
||||
let mut write_batch = Vec::new();
|
||||
for (lsn, value) in values {
|
||||
if key.is_rel_block_key() || key.is_slru_block_key() {
|
||||
// This bails out on first error without modifying pending_updates.
|
||||
// That's Ok, cf this function's doc comment.
|
||||
writer.put(key, lsn, &value, ctx).await?;
|
||||
write_batch.push((key, lsn, value));
|
||||
} else {
|
||||
retained_pending_updates
|
||||
.entry(key)
|
||||
@@ -1805,9 +1817,11 @@ impl<'a> DatadirModification<'a> {
|
||||
.push((lsn, value));
|
||||
}
|
||||
}
|
||||
writer.put_batch(write_batch, ctx).await?;
|
||||
}
|
||||
|
||||
self.pending_updates = retained_pending_updates;
|
||||
self.pending_bytes = 0;
|
||||
|
||||
if pending_nblocks != 0 {
|
||||
writer.update_current_logical_size(pending_nblocks * i64::from(BLCKSZ));
|
||||
@@ -1833,17 +1847,20 @@ impl<'a> DatadirModification<'a> {
|
||||
self.pending_nblocks = 0;
|
||||
|
||||
if !self.pending_updates.is_empty() {
|
||||
// The put_batch call below expects expects the inputs to be sorted by Lsn,
|
||||
// so we do that first.
|
||||
let lsn_ordered_batch: VecMap<Lsn, (Key, Value)> = VecMap::from_iter(
|
||||
self.pending_updates
|
||||
.drain()
|
||||
.map(|(key, vals)| vals.into_iter().map(move |(lsn, val)| (lsn, (key, val))))
|
||||
.kmerge_by(|lhs, rhs| lhs.0 < rhs.0),
|
||||
VecMapOrdering::GreaterOrEqual,
|
||||
);
|
||||
// Ordering: the items in this batch do not need to be in any global order, but values for
|
||||
// a particular Key must be in Lsn order relative to one another. InMemoryLayer relies on
|
||||
// this to do efficient updates to its index.
|
||||
let batch: Vec<(Key, Lsn, Value)> = self
|
||||
.pending_updates
|
||||
.drain()
|
||||
.flat_map(|(key, values)| {
|
||||
values
|
||||
.into_iter()
|
||||
.map(move |(lsn, value)| (key, lsn, value))
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
writer.put_batch(lsn_ordered_batch, ctx).await?;
|
||||
writer.put_batch(batch, ctx).await?;
|
||||
}
|
||||
|
||||
if !self.pending_deletions.is_empty() {
|
||||
@@ -1868,6 +1885,8 @@ impl<'a> DatadirModification<'a> {
|
||||
writer.update_directory_entries_count(kind, count as u64);
|
||||
}
|
||||
|
||||
self.pending_bytes = 0;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -1918,6 +1937,10 @@ impl<'a> DatadirModification<'a> {
|
||||
return;
|
||||
}
|
||||
}
|
||||
self.pending_bytes += match &val {
|
||||
Value::Image(inner) => inner.len(),
|
||||
Value::WalRecord(_) => 100, // Rough approximation of typical serialized WalRecord size.
|
||||
};
|
||||
values.push((self.lsn, val));
|
||||
}
|
||||
|
||||
|
||||
@@ -79,6 +79,8 @@ impl EphemeralFile {
|
||||
self.rw.read_blk(blknum, ctx).await
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
// This is a test helper: outside of tests, we are always written do via a pre-serialized batch.
|
||||
pub(crate) async fn write_blob(
|
||||
&mut self,
|
||||
srcbuf: &[u8],
|
||||
@@ -86,17 +88,28 @@ impl EphemeralFile {
|
||||
) -> Result<u64, io::Error> {
|
||||
let pos = self.rw.bytes_written();
|
||||
|
||||
// Write the length field
|
||||
if srcbuf.len() < 0x80 {
|
||||
// short one-byte length header
|
||||
let len_buf = [srcbuf.len() as u8];
|
||||
let mut len_bytes = std::io::Cursor::new(Vec::new());
|
||||
crate::tenant::storage_layer::inmemory_layer::SerializedBatch::write_blob_length(
|
||||
srcbuf.len(),
|
||||
&mut len_bytes,
|
||||
);
|
||||
let len_bytes = len_bytes.into_inner();
|
||||
|
||||
self.rw.write_all_borrowed(&len_buf, ctx).await?;
|
||||
} else {
|
||||
let mut len_buf = u32::to_be_bytes(srcbuf.len() as u32);
|
||||
len_buf[0] |= 0x80;
|
||||
self.rw.write_all_borrowed(&len_buf, ctx).await?;
|
||||
}
|
||||
// Write the length field
|
||||
self.rw.write_all_borrowed(&len_bytes, ctx).await?;
|
||||
|
||||
// Write the payload
|
||||
self.rw.write_all_borrowed(srcbuf, ctx).await?;
|
||||
|
||||
Ok(pos)
|
||||
}
|
||||
|
||||
pub(crate) async fn write_raw(
|
||||
&mut self,
|
||||
srcbuf: &[u8],
|
||||
ctx: &RequestContext,
|
||||
) -> Result<u64, io::Error> {
|
||||
let pos = self.rw.bytes_written();
|
||||
|
||||
// Write the payload
|
||||
self.rw.write_all_borrowed(srcbuf, ctx).await?;
|
||||
|
||||
@@ -29,6 +29,7 @@ use utils::lsn::Lsn;
|
||||
pub use delta_layer::{DeltaLayer, DeltaLayerWriter, ValueRef};
|
||||
pub use image_layer::{ImageLayer, ImageLayerWriter};
|
||||
pub use inmemory_layer::InMemoryLayer;
|
||||
pub use inmemory_layer::SerializedBatch;
|
||||
pub use layer_desc::{PersistentLayerDesc, PersistentLayerKey};
|
||||
pub use layer_name::{DeltaLayerName, ImageLayerName, LayerName};
|
||||
|
||||
|
||||
@@ -36,13 +36,13 @@ use crate::tenant::block_io::{BlockBuf, BlockCursor, BlockLease, BlockReader, Fi
|
||||
use crate::tenant::disk_btree::{
|
||||
DiskBtreeBuilder, DiskBtreeIterator, DiskBtreeReader, VisitDirection,
|
||||
};
|
||||
use crate::tenant::storage_layer::{Layer, ValueReconstructResult, ValueReconstructState};
|
||||
use crate::tenant::storage_layer::{ValueReconstructResult, ValueReconstructState};
|
||||
use crate::tenant::timeline::GetVectoredError;
|
||||
use crate::tenant::vectored_blob_io::{
|
||||
BlobFlag, MaxVectoredReadBytes, StreamingVectoredReadPlanner, VectoredBlobReader, VectoredRead,
|
||||
VectoredReadPlanner,
|
||||
};
|
||||
use crate::tenant::{PageReconstructError, Timeline};
|
||||
use crate::tenant::PageReconstructError;
|
||||
use crate::virtual_file::{self, VirtualFile};
|
||||
use crate::{walrecord, TEMP_FILE_SUFFIX};
|
||||
use crate::{DELTA_FILE_MAGIC, STORAGE_FORMAT_VERSION};
|
||||
@@ -73,8 +73,7 @@ use utils::{
|
||||
};
|
||||
|
||||
use super::{
|
||||
AsLayerDesc, LayerAccessStats, LayerName, PersistentLayerDesc, ResidentLayer,
|
||||
ValuesReconstructState,
|
||||
AsLayerDesc, LayerAccessStats, LayerName, PersistentLayerDesc, ValuesReconstructState,
|
||||
};
|
||||
|
||||
///
|
||||
@@ -373,7 +372,6 @@ impl DeltaLayer {
|
||||
/// 3. Call `finish`.
|
||||
///
|
||||
struct DeltaLayerWriterInner {
|
||||
conf: &'static PageServerConf,
|
||||
pub path: Utf8PathBuf,
|
||||
timeline_id: TimelineId,
|
||||
tenant_shard_id: TenantShardId,
|
||||
@@ -420,7 +418,6 @@ impl DeltaLayerWriterInner {
|
||||
let tree_builder = DiskBtreeBuilder::new(block_buf);
|
||||
|
||||
Ok(Self {
|
||||
conf,
|
||||
path,
|
||||
timeline_id,
|
||||
tenant_shard_id,
|
||||
@@ -495,11 +492,10 @@ impl DeltaLayerWriterInner {
|
||||
async fn finish(
|
||||
self,
|
||||
key_end: Key,
|
||||
timeline: &Arc<Timeline>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<ResidentLayer> {
|
||||
) -> anyhow::Result<(PersistentLayerDesc, Utf8PathBuf)> {
|
||||
let temp_path = self.path.clone();
|
||||
let result = self.finish0(key_end, timeline, ctx).await;
|
||||
let result = self.finish0(key_end, ctx).await;
|
||||
if result.is_err() {
|
||||
tracing::info!(%temp_path, "cleaning up temporary file after error during writing");
|
||||
if let Err(e) = std::fs::remove_file(&temp_path) {
|
||||
@@ -512,9 +508,8 @@ impl DeltaLayerWriterInner {
|
||||
async fn finish0(
|
||||
self,
|
||||
key_end: Key,
|
||||
timeline: &Arc<Timeline>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<ResidentLayer> {
|
||||
) -> anyhow::Result<(PersistentLayerDesc, Utf8PathBuf)> {
|
||||
let index_start_blk =
|
||||
((self.blob_writer.size() + PAGE_SZ as u64 - 1) / PAGE_SZ as u64) as u32;
|
||||
|
||||
@@ -579,11 +574,9 @@ impl DeltaLayerWriterInner {
|
||||
// fsync the file
|
||||
file.sync_all().await?;
|
||||
|
||||
let layer = Layer::finish_creating(self.conf, timeline, desc, &self.path)?;
|
||||
trace!("created delta layer {}", self.path);
|
||||
|
||||
trace!("created delta layer {}", layer.local_path());
|
||||
|
||||
Ok(layer)
|
||||
Ok((desc, self.path))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -684,14 +677,9 @@ impl DeltaLayerWriter {
|
||||
pub(crate) async fn finish(
|
||||
mut self,
|
||||
key_end: Key,
|
||||
timeline: &Arc<Timeline>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<ResidentLayer> {
|
||||
self.inner
|
||||
.take()
|
||||
.unwrap()
|
||||
.finish(key_end, timeline, ctx)
|
||||
.await
|
||||
) -> anyhow::Result<(PersistentLayerDesc, Utf8PathBuf)> {
|
||||
self.inner.take().unwrap().finish(key_end, ctx).await
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@@ -1687,8 +1675,9 @@ pub(crate) mod test {
|
||||
use super::*;
|
||||
use crate::repository::Value;
|
||||
use crate::tenant::harness::TIMELINE_ID;
|
||||
use crate::tenant::storage_layer::{Layer, ResidentLayer};
|
||||
use crate::tenant::vectored_blob_io::StreamingVectoredReadPlanner;
|
||||
use crate::tenant::Tenant;
|
||||
use crate::tenant::{Tenant, Timeline};
|
||||
use crate::{
|
||||
context::DownloadBehavior,
|
||||
task_mgr::TaskKind,
|
||||
@@ -1982,9 +1971,8 @@ pub(crate) mod test {
|
||||
res?;
|
||||
}
|
||||
|
||||
let resident = writer
|
||||
.finish(entries_meta.key_range.end, &timeline, &ctx)
|
||||
.await?;
|
||||
let (desc, path) = writer.finish(entries_meta.key_range.end, &ctx).await?;
|
||||
let resident = Layer::finish_creating(harness.conf, &timeline, desc, &path)?;
|
||||
|
||||
let inner = resident.get_as_delta(&ctx).await?;
|
||||
|
||||
@@ -2173,7 +2161,8 @@ pub(crate) mod test {
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let copied_layer = writer.finish(Key::MAX, &branch, ctx).await.unwrap();
|
||||
let (desc, path) = writer.finish(Key::MAX, ctx).await.unwrap();
|
||||
let copied_layer = Layer::finish_creating(tenant.conf, &branch, desc, &path).unwrap();
|
||||
|
||||
copied_layer.get_as_delta(ctx).await.unwrap();
|
||||
|
||||
@@ -2301,7 +2290,9 @@ pub(crate) mod test {
|
||||
for (key, lsn, value) in deltas {
|
||||
writer.put_value(key, lsn, value, ctx).await?;
|
||||
}
|
||||
let delta_layer = writer.finish(key_end, tline, ctx).await?;
|
||||
|
||||
let (desc, path) = writer.finish(key_end, ctx).await?;
|
||||
let delta_layer = Layer::finish_creating(tenant.conf, tline, desc, &path)?;
|
||||
|
||||
Ok::<_, anyhow::Error>(delta_layer)
|
||||
}
|
||||
|
||||
@@ -12,9 +12,10 @@ use crate::tenant::block_io::{BlockCursor, BlockReader, BlockReaderRef};
|
||||
use crate::tenant::ephemeral_file::EphemeralFile;
|
||||
use crate::tenant::storage_layer::ValueReconstructResult;
|
||||
use crate::tenant::timeline::GetVectoredError;
|
||||
use crate::tenant::{PageReconstructError, Timeline};
|
||||
use crate::tenant::PageReconstructError;
|
||||
use crate::{l0_flush, page_cache, walrecord};
|
||||
use anyhow::{anyhow, ensure, Result};
|
||||
use camino::Utf8PathBuf;
|
||||
use pageserver_api::keyspace::KeySpace;
|
||||
use pageserver_api::models::InMemoryLayerInfo;
|
||||
use pageserver_api::shard::TenantShardId;
|
||||
@@ -31,16 +32,46 @@ use std::fmt::Write;
|
||||
use std::ops::Range;
|
||||
use std::sync::atomic::Ordering as AtomicOrdering;
|
||||
use std::sync::atomic::{AtomicU64, AtomicUsize};
|
||||
use tokio::sync::{RwLock, RwLockWriteGuard};
|
||||
use tokio::sync::RwLock;
|
||||
|
||||
use super::{
|
||||
DeltaLayerWriter, ResidentLayer, ValueReconstructSituation, ValueReconstructState,
|
||||
DeltaLayerWriter, PersistentLayerDesc, ValueReconstructSituation, ValueReconstructState,
|
||||
ValuesReconstructState,
|
||||
};
|
||||
|
||||
#[derive(Debug, PartialEq, Eq, Clone, Copy, Hash)]
|
||||
pub(crate) struct InMemoryLayerFileId(page_cache::FileId);
|
||||
|
||||
#[derive(Ord, PartialOrd, Eq, PartialEq)]
|
||||
struct IndexPrefix {
|
||||
field1: u8,
|
||||
field2: u32,
|
||||
field3: u32,
|
||||
field4: u32,
|
||||
field5: u8,
|
||||
}
|
||||
|
||||
fn materialize_key(prefix: &IndexPrefix, blkno: u32) -> Key {
|
||||
Key {
|
||||
field1: prefix.field1,
|
||||
field2: prefix.field2,
|
||||
field3: prefix.field3,
|
||||
field4: prefix.field4,
|
||||
field5: prefix.field5,
|
||||
field6: blkno,
|
||||
}
|
||||
}
|
||||
|
||||
fn key_to_prefix(key: &Key) -> IndexPrefix {
|
||||
IndexPrefix {
|
||||
field1: key.field1,
|
||||
field2: key.field2,
|
||||
field3: key.field3,
|
||||
field4: key.field4,
|
||||
field5: key.field5,
|
||||
}
|
||||
}
|
||||
|
||||
pub struct InMemoryLayer {
|
||||
conf: &'static PageServerConf,
|
||||
tenant_shard_id: TenantShardId,
|
||||
@@ -82,7 +113,7 @@ pub struct InMemoryLayerInner {
|
||||
/// All versions of all pages in the layer are kept here. Indexed
|
||||
/// by block number and LSN. The value is an offset into the
|
||||
/// ephemeral file where the page version is stored.
|
||||
index: BTreeMap<Key, VecMap<Lsn, u64>>,
|
||||
index: BTreeMap<IndexPrefix, BTreeMap<u32, VecMap<Lsn, u64>>>,
|
||||
|
||||
/// The values are stored in a serialized format in this file.
|
||||
/// Each serialized Value is preceded by a 'u32' length field.
|
||||
@@ -273,30 +304,34 @@ impl InMemoryLayer {
|
||||
|
||||
let cursor = inner.file.block_cursor();
|
||||
let mut buf = Vec::new();
|
||||
for (key, vec_map) in inner.index.iter() {
|
||||
for (lsn, pos) in vec_map.as_slice() {
|
||||
let mut desc = String::new();
|
||||
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
|
||||
let val = Value::des(&buf);
|
||||
match val {
|
||||
Ok(Value::Image(img)) => {
|
||||
write!(&mut desc, " img {} bytes", img.len())?;
|
||||
}
|
||||
Ok(Value::WalRecord(rec)) => {
|
||||
let wal_desc = walrecord::describe_wal_record(&rec).unwrap();
|
||||
write!(
|
||||
&mut desc,
|
||||
" rec {} bytes will_init: {} {}",
|
||||
buf.len(),
|
||||
rec.will_init(),
|
||||
wal_desc
|
||||
)?;
|
||||
}
|
||||
Err(err) => {
|
||||
write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?;
|
||||
for (key_prefix, inner) in inner.index.iter() {
|
||||
for (blkno, vec_map) in inner {
|
||||
let key = materialize_key(key_prefix, *blkno);
|
||||
|
||||
for (lsn, pos) in vec_map.as_slice() {
|
||||
let mut desc = String::new();
|
||||
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
|
||||
let val = Value::des(&buf);
|
||||
match val {
|
||||
Ok(Value::Image(img)) => {
|
||||
write!(&mut desc, " img {} bytes", img.len())?;
|
||||
}
|
||||
Ok(Value::WalRecord(rec)) => {
|
||||
let wal_desc = walrecord::describe_wal_record(&rec).unwrap();
|
||||
write!(
|
||||
&mut desc,
|
||||
" rec {} bytes will_init: {} {}",
|
||||
buf.len(),
|
||||
rec.will_init(),
|
||||
wal_desc
|
||||
)?;
|
||||
}
|
||||
Err(err) => {
|
||||
write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?;
|
||||
}
|
||||
}
|
||||
println!(" key {} at {}: {}", key, lsn, desc);
|
||||
}
|
||||
println!(" key {} at {}: {}", key, lsn, desc);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -323,23 +358,25 @@ impl InMemoryLayer {
|
||||
let reader = inner.file.block_cursor();
|
||||
|
||||
// Scan the page versions backwards, starting from `lsn`.
|
||||
if let Some(vec_map) = inner.index.get(&key) {
|
||||
let slice = vec_map.slice_range(lsn_range);
|
||||
for (entry_lsn, pos) in slice.iter().rev() {
|
||||
let buf = reader.read_blob(*pos, &ctx).await?;
|
||||
let value = Value::des(&buf)?;
|
||||
match value {
|
||||
Value::Image(img) => {
|
||||
reconstruct_state.img = Some((*entry_lsn, img));
|
||||
return Ok(ValueReconstructResult::Complete);
|
||||
}
|
||||
Value::WalRecord(rec) => {
|
||||
let will_init = rec.will_init();
|
||||
reconstruct_state.records.push((*entry_lsn, rec));
|
||||
if will_init {
|
||||
// This WAL record initializes the page, so no need to go further back
|
||||
need_image = false;
|
||||
break;
|
||||
if let Some(inner) = inner.index.get(&key_to_prefix(&key)) {
|
||||
if let Some(vec_map) = inner.get(&key.field6) {
|
||||
let slice = vec_map.slice_range(lsn_range);
|
||||
for (entry_lsn, pos) in slice.iter().rev() {
|
||||
let buf = reader.read_blob(*pos, &ctx).await?;
|
||||
let value = Value::des(&buf)?;
|
||||
match value {
|
||||
Value::Image(img) => {
|
||||
reconstruct_state.img = Some((*entry_lsn, img));
|
||||
return Ok(ValueReconstructResult::Complete);
|
||||
}
|
||||
Value::WalRecord(rec) => {
|
||||
let will_init = rec.will_init();
|
||||
reconstruct_state.records.push((*entry_lsn, rec));
|
||||
if will_init {
|
||||
// This WAL record initializes the page, so no need to go further back
|
||||
need_image = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -376,34 +413,54 @@ impl InMemoryLayer {
|
||||
let reader = inner.file.block_cursor();
|
||||
|
||||
for range in keyspace.ranges.iter() {
|
||||
for (key, vec_map) in inner.index.range(range.start..range.end) {
|
||||
let lsn_range = match reconstruct_state.get_cached_lsn(key) {
|
||||
Some(cached_lsn) => (cached_lsn + 1)..end_lsn,
|
||||
None => self.start_lsn..end_lsn,
|
||||
let range_incl = range.start..=Key::from_i128(Key::to_i128(&range.end) - 1);
|
||||
|
||||
let prefix_start = key_to_prefix(&range.start);
|
||||
let prefix_end = key_to_prefix(&range.end);
|
||||
|
||||
for (prefix, relation_idx) in inner.index.range(prefix_start..=prefix_end) {
|
||||
let blkno_start = if prefix == &key_to_prefix(&range_incl.start()) {
|
||||
range_incl.start().field6
|
||||
} else {
|
||||
0
|
||||
};
|
||||
|
||||
let slice = vec_map.slice_range(lsn_range);
|
||||
let blkno_end = if prefix == &key_to_prefix(&range_incl.end()) {
|
||||
range_incl.end().field6
|
||||
} else {
|
||||
0xffffffff
|
||||
};
|
||||
|
||||
for (entry_lsn, pos) in slice.iter().rev() {
|
||||
// TODO: this uses the page cache => https://github.com/neondatabase/neon/issues/8183
|
||||
let buf = reader.read_blob(*pos, &ctx).await;
|
||||
if let Err(e) = buf {
|
||||
reconstruct_state
|
||||
.on_key_error(*key, PageReconstructError::from(anyhow!(e)));
|
||||
break;
|
||||
}
|
||||
for (blkno, vec_map) in relation_idx.range(blkno_start..=blkno_end) {
|
||||
let key = materialize_key(prefix, *blkno);
|
||||
let lsn_range = match reconstruct_state.get_cached_lsn(&key) {
|
||||
Some(cached_lsn) => (cached_lsn + 1)..end_lsn,
|
||||
None => self.start_lsn..end_lsn,
|
||||
};
|
||||
|
||||
let value = Value::des(&buf.unwrap());
|
||||
if let Err(e) = value {
|
||||
reconstruct_state
|
||||
.on_key_error(*key, PageReconstructError::from(anyhow!(e)));
|
||||
break;
|
||||
}
|
||||
let slice = vec_map.slice_range(lsn_range);
|
||||
|
||||
let key_situation =
|
||||
reconstruct_state.update_key(key, *entry_lsn, value.unwrap());
|
||||
if key_situation == ValueReconstructSituation::Complete {
|
||||
break;
|
||||
for (entry_lsn, pos) in slice.iter().rev() {
|
||||
// TODO: this uses the page cache => https://github.com/neondatabase/neon/issues/8183
|
||||
let buf = reader.read_blob(*pos, &ctx).await;
|
||||
if let Err(e) = buf {
|
||||
reconstruct_state
|
||||
.on_key_error(key, PageReconstructError::from(anyhow!(e)));
|
||||
break;
|
||||
}
|
||||
|
||||
let value = Value::des(&buf.unwrap());
|
||||
if let Err(e) = value {
|
||||
reconstruct_state
|
||||
.on_key_error(key, PageReconstructError::from(anyhow!(e)));
|
||||
break;
|
||||
}
|
||||
|
||||
let key_situation =
|
||||
reconstruct_state.update_key(&key, *entry_lsn, value.unwrap());
|
||||
if key_situation == ValueReconstructSituation::Complete {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -415,6 +472,74 @@ impl InMemoryLayer {
|
||||
}
|
||||
}
|
||||
|
||||
pub struct SerializedBatch {
|
||||
/// Blobs serialized in EphemeralFile's native format, ready for passing to [`EphemeralFile::write_raw`].
|
||||
pub(crate) raw: Vec<u8>,
|
||||
|
||||
/// Index of values in [`Self::raw`], using offsets relative to the start of the buffer.
|
||||
pub(crate) offsets: Vec<(Key, Lsn, u64)>,
|
||||
|
||||
/// The highest LSN of any value in the batch
|
||||
pub(crate) max_lsn: Lsn,
|
||||
}
|
||||
|
||||
impl SerializedBatch {
|
||||
/// Write a blob length in the internal format of the EphemeralFile
|
||||
pub(crate) fn write_blob_length(len: usize, cursor: &mut std::io::Cursor<Vec<u8>>) {
|
||||
use std::io::Write;
|
||||
|
||||
if len < 0x80 {
|
||||
// short one-byte length header
|
||||
let len_buf = [len as u8];
|
||||
|
||||
cursor
|
||||
.write_all(&len_buf)
|
||||
.expect("Writing to Vec is infallible");
|
||||
} else {
|
||||
let mut len_buf = u32::to_be_bytes(len as u32);
|
||||
len_buf[0] |= 0x80;
|
||||
cursor
|
||||
.write_all(&len_buf)
|
||||
.expect("Writing to Vec is infallible");
|
||||
}
|
||||
}
|
||||
|
||||
pub fn from_values(batch: Vec<(Key, Lsn, Value)>) -> Self {
|
||||
use std::io::Write;
|
||||
|
||||
let mut offsets: Vec<(Key, Lsn, u64)> = Vec::new();
|
||||
let mut cursor = std::io::Cursor::new(Vec::<u8>::with_capacity(batch.len() * 8192));
|
||||
let mut max_lsn: Lsn = Lsn(0);
|
||||
let mut value_buf = smallvec::SmallVec::<[u8; 256]>::new();
|
||||
for (key, lsn, val) in batch {
|
||||
let relative_off = cursor.position();
|
||||
|
||||
value_buf.clear();
|
||||
val.ser_into(&mut value_buf)
|
||||
.expect("Value serialization is infallible");
|
||||
Self::write_blob_length(value_buf.len(), &mut cursor);
|
||||
|
||||
cursor
|
||||
.write_all(&value_buf)
|
||||
.expect("Writing to Vec is infallible");
|
||||
|
||||
// We can't write straight into the buffer, because the InMemoryLayer file format requires
|
||||
// the size to come before the value. However... we could probably calculate the size before
|
||||
// actually serializing the value
|
||||
//val.ser_into(&mut cursor)?;
|
||||
|
||||
offsets.push((key, lsn, relative_off));
|
||||
max_lsn = std::cmp::max(max_lsn, lsn);
|
||||
}
|
||||
|
||||
Self {
|
||||
raw: cursor.into_inner(),
|
||||
offsets,
|
||||
max_lsn,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn inmem_layer_display(mut f: impl Write, start_lsn: Lsn, end_lsn: Lsn) -> std::fmt::Result {
|
||||
write!(f, "inmem-{:016X}-{:016X}", start_lsn.0, end_lsn.0)
|
||||
}
|
||||
@@ -478,38 +603,20 @@ impl InMemoryLayer {
|
||||
})
|
||||
}
|
||||
|
||||
// Write operations
|
||||
|
||||
/// Common subroutine of the public put_wal_record() and put_page_image() functions.
|
||||
/// Adds the page version to the in-memory tree
|
||||
|
||||
pub(crate) async fn put_value(
|
||||
// Write path.
|
||||
pub async fn put_batch(
|
||||
&self,
|
||||
key: Key,
|
||||
lsn: Lsn,
|
||||
buf: &[u8],
|
||||
serialized_batch: &SerializedBatch,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<()> {
|
||||
let mut inner = self.inner.write().await;
|
||||
self.assert_writable();
|
||||
self.put_value_locked(&mut inner, key, lsn, buf, ctx).await
|
||||
}
|
||||
//self.assert_writable();
|
||||
|
||||
async fn put_value_locked(
|
||||
&self,
|
||||
locked_inner: &mut RwLockWriteGuard<'_, InMemoryLayerInner>,
|
||||
key: Key,
|
||||
lsn: Lsn,
|
||||
buf: &[u8],
|
||||
ctx: &RequestContext,
|
||||
) -> Result<()> {
|
||||
trace!("put_value key {} at {}/{}", key, self.timeline_id, lsn);
|
||||
|
||||
let off = {
|
||||
locked_inner
|
||||
let base_off = {
|
||||
inner
|
||||
.file
|
||||
.write_blob(
|
||||
buf,
|
||||
.write_raw(
|
||||
&serialized_batch.raw,
|
||||
&RequestContextBuilder::extend(ctx)
|
||||
.page_content_kind(PageContentKind::InMemoryLayer)
|
||||
.build(),
|
||||
@@ -517,15 +624,21 @@ impl InMemoryLayer {
|
||||
.await?
|
||||
};
|
||||
|
||||
let vec_map = locked_inner.index.entry(key).or_default();
|
||||
let old = vec_map.append_or_update_last(lsn, off).unwrap().0;
|
||||
if old.is_some() {
|
||||
// We already had an entry for this LSN. That's odd..
|
||||
warn!("Key {} at {} already exists", key, lsn);
|
||||
for (key, lsn, relative_off) in &serialized_batch.offsets {
|
||||
let prefix = key_to_prefix(&key);
|
||||
|
||||
let relation_idx = match inner.index.get_mut(&prefix) {
|
||||
Some(i) => i,
|
||||
None => inner.index.entry(prefix).or_default(),
|
||||
};
|
||||
|
||||
let off = base_off + relative_off;
|
||||
let vec_map = relation_idx.entry(key.field6).or_default();
|
||||
vec_map.append_fast(*lsn, off);
|
||||
}
|
||||
|
||||
let size = locked_inner.file.len();
|
||||
locked_inner.resource_units.maybe_publish_size(size);
|
||||
let size = inner.file.len();
|
||||
inner.resource_units.maybe_publish_size(size);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
@@ -548,8 +661,6 @@ impl InMemoryLayer {
|
||||
/// Records the end_lsn for non-dropped layers.
|
||||
/// `end_lsn` is exclusive
|
||||
pub async fn freeze(&self, end_lsn: Lsn) {
|
||||
let inner = self.inner.write().await;
|
||||
|
||||
assert!(
|
||||
self.start_lsn < end_lsn,
|
||||
"{} >= {}",
|
||||
@@ -567,11 +678,15 @@ impl InMemoryLayer {
|
||||
})
|
||||
.expect("frozen_local_path_str set only once");
|
||||
|
||||
for vec_map in inner.index.values() {
|
||||
for (lsn, _pos) in vec_map.as_slice() {
|
||||
assert!(*lsn < end_lsn);
|
||||
}
|
||||
}
|
||||
// #[cfg(debug_assertions)]
|
||||
// {
|
||||
// let inner = self.inner.write().await;
|
||||
// for vec_map in inner.index.values() {
|
||||
// for (lsn, _pos) in vec_map.as_slice() {
|
||||
// assert!(*lsn < end_lsn);
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
}
|
||||
|
||||
/// Write this frozen in-memory layer to disk. If `key_range` is set, the delta
|
||||
@@ -579,12 +694,12 @@ impl InMemoryLayer {
|
||||
/// if there are no matching keys.
|
||||
///
|
||||
/// Returns a new delta layer with all the same data as this in-memory layer
|
||||
pub(crate) async fn write_to_disk(
|
||||
pub async fn write_to_disk(
|
||||
&self,
|
||||
timeline: &Arc<Timeline>,
|
||||
ctx: &RequestContext,
|
||||
key_range: Option<Range<Key>>,
|
||||
) -> Result<Option<ResidentLayer>> {
|
||||
l0_flush_global_state: &l0_flush::Inner,
|
||||
) -> Result<Option<(PersistentLayerDesc, Utf8PathBuf)>> {
|
||||
// Grab the lock in read-mode. We hold it over the I/O, but because this
|
||||
// layer is not writeable anymore, no one should be trying to acquire the
|
||||
// write lock on it, so we shouldn't block anyone. There's one exception
|
||||
@@ -596,9 +711,8 @@ impl InMemoryLayer {
|
||||
// rare though, so we just accept the potential latency hit for now.
|
||||
let inner = self.inner.read().await;
|
||||
|
||||
let l0_flush_global_state = timeline.l0_flush_global_state.inner().clone();
|
||||
use l0_flush::Inner;
|
||||
let _concurrency_permit = match &*l0_flush_global_state {
|
||||
let _concurrency_permit = match l0_flush_global_state {
|
||||
Inner::PageCached => None,
|
||||
Inner::Direct { semaphore, .. } => Some(semaphore.acquire().await),
|
||||
};
|
||||
@@ -606,11 +720,12 @@ impl InMemoryLayer {
|
||||
let end_lsn = *self.end_lsn.get().unwrap();
|
||||
|
||||
let key_count = if let Some(key_range) = key_range {
|
||||
inner
|
||||
.index
|
||||
.iter()
|
||||
.filter(|(k, _)| key_range.contains(k))
|
||||
.count()
|
||||
panic!("Update for IndexPrefix");
|
||||
// inner
|
||||
// .index
|
||||
// .iter()
|
||||
// .filter(|(k, _)| key_range.contains(k))
|
||||
// .count()
|
||||
} else {
|
||||
inner.index.len()
|
||||
};
|
||||
@@ -628,7 +743,7 @@ impl InMemoryLayer {
|
||||
)
|
||||
.await?;
|
||||
|
||||
match &*l0_flush_global_state {
|
||||
match l0_flush_global_state {
|
||||
l0_flush::Inner::PageCached => {
|
||||
let ctx = RequestContextBuilder::extend(ctx)
|
||||
.page_content_kind(PageContentKind::InMemoryLayer)
|
||||
@@ -638,16 +753,20 @@ impl InMemoryLayer {
|
||||
|
||||
let cursor = inner.file.block_cursor();
|
||||
|
||||
for (key, vec_map) in inner.index.iter() {
|
||||
// Write all page versions
|
||||
for (lsn, pos) in vec_map.as_slice() {
|
||||
cursor.read_blob_into_buf(*pos, &mut buf, &ctx).await?;
|
||||
let will_init = Value::des(&buf)?.will_init();
|
||||
let res;
|
||||
(buf, res) = delta_layer_writer
|
||||
.put_value_bytes(*key, *lsn, buf, will_init, &ctx)
|
||||
.await;
|
||||
res?;
|
||||
for (key_prefix, inner) in inner.index.iter() {
|
||||
for (blkno, vec_map) in inner {
|
||||
let key = materialize_key(key_prefix, *blkno);
|
||||
|
||||
// Write all page versions
|
||||
for (lsn, pos) in vec_map.as_slice() {
|
||||
cursor.read_blob_into_buf(*pos, &mut buf, &ctx).await?;
|
||||
let will_init = Value::des(&buf)?.will_init();
|
||||
let res;
|
||||
(buf, res) = delta_layer_writer
|
||||
.put_value_bytes(key, *lsn, buf, will_init, &ctx)
|
||||
.await;
|
||||
res?;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -671,29 +790,32 @@ impl InMemoryLayer {
|
||||
|
||||
let mut buf = Vec::new();
|
||||
|
||||
for (key, vec_map) in inner.index.iter() {
|
||||
// Write all page versions
|
||||
for (lsn, pos) in vec_map.as_slice() {
|
||||
// TODO: once we have blob lengths in the in-memory index, we can
|
||||
// 1. get rid of the blob_io / BlockReaderRef::Slice business and
|
||||
// 2. load the file contents into a Bytes and
|
||||
// 3. the use `Bytes::slice` to get the `buf` that is our blob
|
||||
// 4. pass that `buf` into `put_value_bytes`
|
||||
// => https://github.com/neondatabase/neon/issues/8183
|
||||
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
|
||||
let will_init = Value::des(&buf)?.will_init();
|
||||
let res;
|
||||
(buf, res) = delta_layer_writer
|
||||
.put_value_bytes(*key, *lsn, buf, will_init, ctx)
|
||||
.await;
|
||||
res?;
|
||||
for (key_prefix, inner) in inner.index.iter() {
|
||||
for (blkno, vec_map) in inner {
|
||||
// Write all page versions
|
||||
let key = materialize_key(key_prefix, *blkno);
|
||||
for (lsn, pos) in vec_map.as_slice() {
|
||||
// TODO: once we have blob lengths in the in-memory index, we can
|
||||
// 1. get rid of the blob_io / BlockReaderRef::Slice business and
|
||||
// 2. load the file contents into a Bytes and
|
||||
// 3. the use `Bytes::slice` to get the `buf` that is our blob
|
||||
// 4. pass that `buf` into `put_value_bytes`
|
||||
// => https://github.com/neondatabase/neon/issues/8183
|
||||
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
|
||||
let will_init = Value::des(&buf)?.will_init();
|
||||
let res;
|
||||
(buf, res) = delta_layer_writer
|
||||
.put_value_bytes(key, *lsn, buf, will_init, ctx)
|
||||
.await;
|
||||
res?;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// MAX is used here because we identify L0 layers by full key range
|
||||
let delta_layer = delta_layer_writer.finish(Key::MAX, timeline, ctx).await?;
|
||||
let (desc, path) = delta_layer_writer.finish(Key::MAX, ctx).await?;
|
||||
|
||||
// Hold the permit until all the IO is done, including the fsync in `delta_layer_writer.finish()``.
|
||||
//
|
||||
@@ -705,6 +827,6 @@ impl InMemoryLayer {
|
||||
// we dirtied when writing to the filesystem have been flushed and marked !dirty.
|
||||
drop(_concurrency_permit);
|
||||
|
||||
Ok(Some(delta_layer))
|
||||
Ok(Some((desc, path)))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,6 +4,7 @@ use bytes::Bytes;
|
||||
use pageserver_api::key::{Key, KEY_SIZE};
|
||||
use utils::{id::TimelineId, lsn::Lsn, shard::TenantShardId};
|
||||
|
||||
use crate::tenant::storage_layer::Layer;
|
||||
use crate::{config::PageServerConf, context::RequestContext, repository::Value, tenant::Timeline};
|
||||
|
||||
use super::{DeltaLayerWriter, ImageLayerWriter, ResidentLayer};
|
||||
@@ -173,8 +174,9 @@ impl SplitDeltaLayerWriter {
|
||||
)
|
||||
.await?;
|
||||
let prev_delta_writer = std::mem::replace(&mut self.inner, next_delta_writer);
|
||||
self.generated_layers
|
||||
.push(prev_delta_writer.finish(key, tline, ctx).await?);
|
||||
let (desc, path) = prev_delta_writer.finish(key, ctx).await?;
|
||||
let delta_layer = Layer::finish_creating(self.conf, tline, desc, &path)?;
|
||||
self.generated_layers.push(delta_layer);
|
||||
}
|
||||
self.inner.put_value(key, lsn, val, ctx).await
|
||||
}
|
||||
@@ -190,7 +192,10 @@ impl SplitDeltaLayerWriter {
|
||||
inner,
|
||||
..
|
||||
} = self;
|
||||
generated_layers.push(inner.finish(end_key, tline, ctx).await?);
|
||||
|
||||
let (desc, path) = inner.finish(end_key, ctx).await?;
|
||||
let delta_layer = Layer::finish_creating(self.conf, tline, desc, &path)?;
|
||||
generated_layers.push(delta_layer);
|
||||
Ok(generated_layers)
|
||||
}
|
||||
|
||||
|
||||
@@ -47,7 +47,6 @@ use utils::{
|
||||
bin_ser::BeSer,
|
||||
fs_ext, pausable_failpoint,
|
||||
sync::gate::{Gate, GateGuard},
|
||||
vec_map::VecMap,
|
||||
};
|
||||
|
||||
use std::pin::pin;
|
||||
@@ -140,7 +139,7 @@ use self::layer_manager::LayerManager;
|
||||
use self::logical_size::LogicalSize;
|
||||
use self::walreceiver::{WalReceiver, WalReceiverConf};
|
||||
|
||||
use super::{config::TenantConf, upload_queue::NotInitialized};
|
||||
use super::{config::TenantConf, storage_layer::inmemory_layer, upload_queue::NotInitialized};
|
||||
use super::{debug_assert_current_span_has_tenant_and_timeline_id, AttachedTenantConf};
|
||||
use super::{remote_timeline_client::index::IndexPart, storage_layer::LayerFringe};
|
||||
use super::{
|
||||
@@ -4199,12 +4198,14 @@ impl Timeline {
|
||||
let frozen_layer = Arc::clone(frozen_layer);
|
||||
let ctx = ctx.attached_child();
|
||||
let work = async move {
|
||||
let Some(new_delta) = frozen_layer
|
||||
.write_to_disk(&self_clone, &ctx, key_range)
|
||||
let Some((desc, path)) = frozen_layer
|
||||
.write_to_disk(&ctx, key_range, self_clone.l0_flush_global_state.inner())
|
||||
.await?
|
||||
else {
|
||||
return Ok(None);
|
||||
};
|
||||
let new_delta = Layer::finish_creating(self_clone.conf, &self_clone, desc, &path)?;
|
||||
|
||||
// The write_to_disk() above calls writer.finish() which already did the fsync of the inodes.
|
||||
// We just need to fsync the directory in which these inodes are linked,
|
||||
// which we know to be the timeline directory.
|
||||
@@ -5815,9 +5816,8 @@ impl Timeline {
|
||||
for (key, lsn, val) in deltas.data {
|
||||
delta_layer_writer.put_value(key, lsn, val, ctx).await?;
|
||||
}
|
||||
let delta_layer = delta_layer_writer
|
||||
.finish(deltas.key_range.end, self, ctx)
|
||||
.await?;
|
||||
let (desc, path) = delta_layer_writer.finish(deltas.key_range.end, ctx).await?;
|
||||
let delta_layer = Layer::finish_creating(self.conf, self, desc, &path)?;
|
||||
|
||||
{
|
||||
let mut guard = self.layers.write().await;
|
||||
@@ -5931,44 +5931,6 @@ enum OpenLayerAction {
|
||||
}
|
||||
|
||||
impl<'a> TimelineWriter<'a> {
|
||||
/// Put a new page version that can be constructed from a WAL record
|
||||
///
|
||||
/// This will implicitly extend the relation, if the page is beyond the
|
||||
/// current end-of-file.
|
||||
pub(crate) async fn put(
|
||||
&mut self,
|
||||
key: Key,
|
||||
lsn: Lsn,
|
||||
value: &Value,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// Avoid doing allocations for "small" values.
|
||||
// In the regression test suite, the limit of 256 avoided allocations in 95% of cases:
|
||||
// https://github.com/neondatabase/neon/pull/5056#discussion_r1301975061
|
||||
let mut buf = smallvec::SmallVec::<[u8; 256]>::new();
|
||||
value.ser_into(&mut buf)?;
|
||||
let buf_size: u64 = buf.len().try_into().expect("oversized value buf");
|
||||
|
||||
let action = self.get_open_layer_action(lsn, buf_size);
|
||||
let layer = self.handle_open_layer_action(lsn, action, ctx).await?;
|
||||
let res = layer.put_value(key, lsn, &buf, ctx).await;
|
||||
|
||||
if res.is_ok() {
|
||||
// Update the current size only when the entire write was ok.
|
||||
// In case of failures, we may have had partial writes which
|
||||
// render the size tracking out of sync. That's ok because
|
||||
// the checkpoint distance should be significantly smaller
|
||||
// than the S3 single shot upload limit of 5GiB.
|
||||
let state = self.write_guard.as_mut().unwrap();
|
||||
|
||||
state.current_size += buf_size;
|
||||
state.prev_lsn = Some(lsn);
|
||||
state.max_lsn = std::cmp::max(state.max_lsn, Some(lsn));
|
||||
}
|
||||
|
||||
res
|
||||
}
|
||||
|
||||
async fn handle_open_layer_action(
|
||||
&mut self,
|
||||
at: Lsn,
|
||||
@@ -6071,18 +6033,52 @@ impl<'a> TimelineWriter<'a> {
|
||||
}
|
||||
|
||||
/// Put a batch of keys at the specified Lsns.
|
||||
///
|
||||
/// The batch is sorted by Lsn (enforced by usage of [`utils::vec_map::VecMap`].
|
||||
pub(crate) async fn put_batch(
|
||||
&mut self,
|
||||
batch: VecMap<Lsn, (Key, Value)>,
|
||||
batch: Vec<(Key, Lsn, Value)>,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
for (lsn, (key, val)) in batch {
|
||||
self.put(key, lsn, &val, ctx).await?
|
||||
if batch.is_empty() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
Ok(())
|
||||
let serialized_batch = inmemory_layer::SerializedBatch::from_values(batch);
|
||||
let batch_max_lsn = serialized_batch.max_lsn;
|
||||
let buf_size: u64 = serialized_batch.raw.len() as u64;
|
||||
|
||||
let action = self.get_open_layer_action(batch_max_lsn, buf_size);
|
||||
let layer = self
|
||||
.handle_open_layer_action(batch_max_lsn, action, ctx)
|
||||
.await?;
|
||||
|
||||
let res = layer.put_batch(&serialized_batch, ctx).await;
|
||||
|
||||
if res.is_ok() {
|
||||
// Update the current size only when the entire write was ok.
|
||||
// In case of failures, we may have had partial writes which
|
||||
// render the size tracking out of sync. That's ok because
|
||||
// the checkpoint distance should be significantly smaller
|
||||
// than the S3 single shot upload limit of 5GiB.
|
||||
let state = self.write_guard.as_mut().unwrap();
|
||||
|
||||
state.current_size += buf_size;
|
||||
state.prev_lsn = Some(batch_max_lsn);
|
||||
state.max_lsn = std::cmp::max(state.max_lsn, Some(batch_max_lsn));
|
||||
}
|
||||
|
||||
res
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
/// Test helper, for tests that would like to poke individual values without composing a batch
|
||||
pub(crate) async fn put(
|
||||
&mut self,
|
||||
key: Key,
|
||||
lsn: Lsn,
|
||||
value: &Value,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
self.put_batch(vec![(key, lsn, value.clone())], ctx).await
|
||||
}
|
||||
|
||||
pub(crate) async fn delete_batch(
|
||||
|
||||
@@ -1006,14 +1006,16 @@ impl Timeline {
|
||||
|| contains_hole
|
||||
{
|
||||
// ... if so, flush previous layer and prepare to write new one
|
||||
new_layers.push(
|
||||
writer
|
||||
.take()
|
||||
.unwrap()
|
||||
.finish(prev_key.unwrap().next(), self, ctx)
|
||||
.await
|
||||
.map_err(CompactionError::Other)?,
|
||||
);
|
||||
let (desc, path) = writer
|
||||
.take()
|
||||
.unwrap()
|
||||
.finish(prev_key.unwrap().next(), ctx)
|
||||
.await
|
||||
.map_err(CompactionError::Other)?;
|
||||
let new_delta = Layer::finish_creating(self.conf, self, desc, &path)
|
||||
.map_err(CompactionError::Other)?;
|
||||
|
||||
new_layers.push(new_delta);
|
||||
writer = None;
|
||||
|
||||
if contains_hole {
|
||||
@@ -1076,12 +1078,13 @@ impl Timeline {
|
||||
prev_key = Some(key);
|
||||
}
|
||||
if let Some(writer) = writer {
|
||||
new_layers.push(
|
||||
writer
|
||||
.finish(prev_key.unwrap().next(), self, ctx)
|
||||
.await
|
||||
.map_err(CompactionError::Other)?,
|
||||
);
|
||||
let (desc, path) = writer
|
||||
.finish(prev_key.unwrap().next(), ctx)
|
||||
.await
|
||||
.map_err(CompactionError::Other)?;
|
||||
let new_delta = Layer::finish_creating(self.conf, self, desc, &path)
|
||||
.map_err(CompactionError::Other)?;
|
||||
new_layers.push(new_delta);
|
||||
}
|
||||
|
||||
// Sync layers
|
||||
@@ -1857,9 +1860,11 @@ impl Timeline {
|
||||
for (key, lsn, val) in deltas {
|
||||
delta_layer_writer.put_value(key, lsn, val, ctx).await?;
|
||||
}
|
||||
let delta_layer = delta_layer_writer
|
||||
.finish(delta_key.key_range.end, tline, ctx)
|
||||
|
||||
let (desc, path) = delta_layer_writer
|
||||
.finish(delta_key.key_range.end, ctx)
|
||||
.await?;
|
||||
let delta_layer = Layer::finish_creating(tline.conf, tline, desc, &path)?;
|
||||
Ok(Some(FlushDeltaResult::CreateResidentLayer(delta_layer)))
|
||||
}
|
||||
|
||||
@@ -2268,9 +2273,9 @@ impl CompactionJobExecutor for TimelineAdaptor {
|
||||
))
|
||||
});
|
||||
|
||||
let new_delta_layer = writer
|
||||
.finish(prev.unwrap().0.next(), &self.timeline, ctx)
|
||||
.await?;
|
||||
let (desc, path) = writer.finish(prev.unwrap().0.next(), ctx).await?;
|
||||
let new_delta_layer =
|
||||
Layer::finish_creating(self.timeline.conf, &self.timeline, desc, &path)?;
|
||||
|
||||
self.new_deltas.push(new_delta_layer);
|
||||
Ok(())
|
||||
|
||||
@@ -488,10 +488,12 @@ async fn copy_lsn_prefix(
|
||||
// reuse the key instead of adding more holes between layers by using the real
|
||||
// highest key in the layer.
|
||||
let reused_highest_key = layer.layer_desc().key_range.end;
|
||||
let copied = writer
|
||||
.finish(reused_highest_key, target_timeline, ctx)
|
||||
let (desc, path) = writer
|
||||
.finish(reused_highest_key, ctx)
|
||||
.await
|
||||
.map_err(CopyDeltaPrefix)?;
|
||||
let copied = Layer::finish_creating(target_timeline.conf, target_timeline, desc, &path)
|
||||
.map_err(CopyDeltaPrefix)?;
|
||||
|
||||
tracing::debug!(%layer, %copied, "new layer produced");
|
||||
|
||||
|
||||
@@ -27,8 +27,8 @@ use super::TaskStateUpdate;
|
||||
use crate::{
|
||||
context::RequestContext,
|
||||
metrics::{LIVE_CONNECTIONS, WALRECEIVER_STARTED_CONNECTIONS, WAL_INGEST},
|
||||
task_mgr::TaskKind,
|
||||
task_mgr::WALRECEIVER_RUNTIME,
|
||||
pgdatadir_mapping::DatadirModification,
|
||||
task_mgr::{TaskKind, WALRECEIVER_RUNTIME},
|
||||
tenant::{debug_assert_current_span_has_tenant_and_timeline_id, Timeline, WalReceiverInfo},
|
||||
walingest::WalIngest,
|
||||
walrecord::DecodedWALRecord,
|
||||
@@ -342,7 +342,10 @@ pub(super) async fn handle_walreceiver_connection(
|
||||
// Commit every ingest_batch_size records. Even if we filtered out
|
||||
// all records, we still need to call commit to advance the LSN.
|
||||
uncommitted_records += 1;
|
||||
if uncommitted_records >= ingest_batch_size {
|
||||
if uncommitted_records >= ingest_batch_size
|
||||
|| modification.approx_pending_bytes()
|
||||
> DatadirModification::MAX_PENDING_BYTES
|
||||
{
|
||||
WAL_INGEST
|
||||
.records_committed
|
||||
.inc_by(uncommitted_records - filtered_records);
|
||||
|
||||
@@ -1,6 +1,5 @@
|
||||
from contextlib import closing
|
||||
|
||||
import pytest
|
||||
from fixtures.benchmark_fixture import MetricReport
|
||||
from fixtures.common_types import Lsn
|
||||
from fixtures.compare_fixtures import NeonCompare, PgCompare
|
||||
@@ -17,7 +16,6 @@ from fixtures.pg_version import PgVersion
|
||||
# 3. Disk space used
|
||||
# 4. Peak memory usage
|
||||
#
|
||||
@pytest.mark.skip("See https://github.com/neondatabase/neon/issues/7124")
|
||||
def test_bulk_insert(neon_with_baseline: PgCompare):
|
||||
env = neon_with_baseline
|
||||
|
||||
|
||||
Reference in New Issue
Block a user