mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-04 03:52:56 +00:00
refactor(ephemeral_file): reuse owned_buffers_io::BufferedWriter (#7484)
part of https://github.com/neondatabase/neon/issues/7124 Changes ------- This PR replaces the `EphemeralFile::write_blob`-specifc `struct Writer` with re-use of `owned_buffers_io::write::BufferedWriter`. Further, it restructures the code to cleanly separate * the high-level aspect of EphemeralFile's write_blob / read_blk API * the page-caching aspect * the aspect of IO * performing buffered write IO to an underlying VirtualFile * serving reads from either the VirtualFile or the buffer if it hasn't been flushed yet * the annoying "feature" that reads past the end of the written range are allowed and expected to return zeroed memory, as long as one remains within one PAGE_SZ
This commit is contained in:
committed by
GitHub
parent
bf369f4268
commit
dbb0c967d5
@@ -361,6 +361,8 @@ pub enum TaskKind {
|
||||
|
||||
DebugTool,
|
||||
|
||||
EphemeralFilePreWarmPageCache,
|
||||
|
||||
#[cfg(test)]
|
||||
UnitTest,
|
||||
}
|
||||
|
||||
@@ -3,36 +3,26 @@
|
||||
|
||||
use crate::config::PageServerConf;
|
||||
use crate::context::RequestContext;
|
||||
use crate::page_cache::{self, PAGE_SZ};
|
||||
use crate::page_cache;
|
||||
use crate::tenant::block_io::{BlockCursor, BlockLease, BlockReader};
|
||||
use crate::virtual_file::{self, VirtualFile};
|
||||
use bytes::BytesMut;
|
||||
use camino::Utf8PathBuf;
|
||||
use pageserver_api::shard::TenantShardId;
|
||||
use std::cmp::min;
|
||||
|
||||
use std::io::{self, ErrorKind};
|
||||
use std::ops::DerefMut;
|
||||
use std::io;
|
||||
use std::sync::atomic::AtomicU64;
|
||||
use tracing::*;
|
||||
use utils::id::TimelineId;
|
||||
|
||||
pub struct EphemeralFile {
|
||||
page_cache_file_id: page_cache::FileId,
|
||||
|
||||
_tenant_shard_id: TenantShardId,
|
||||
_timeline_id: TimelineId,
|
||||
file: VirtualFile,
|
||||
len: u64,
|
||||
/// An ephemeral file is append-only.
|
||||
/// We keep the last page, which can still be modified, in [`Self::mutable_tail`].
|
||||
/// The other pages, which can no longer be modified, are accessed through the page cache.
|
||||
///
|
||||
/// None <=> IO is ongoing.
|
||||
/// Size is fixed to PAGE_SZ at creation time and must not be changed.
|
||||
mutable_tail: Option<BytesMut>,
|
||||
|
||||
rw: page_caching::RW,
|
||||
}
|
||||
|
||||
mod page_caching;
|
||||
mod zero_padded_read_write;
|
||||
|
||||
impl EphemeralFile {
|
||||
pub async fn create(
|
||||
conf: &PageServerConf,
|
||||
@@ -59,21 +49,18 @@ impl EphemeralFile {
|
||||
.await?;
|
||||
|
||||
Ok(EphemeralFile {
|
||||
page_cache_file_id: page_cache::next_file_id(),
|
||||
_tenant_shard_id: tenant_shard_id,
|
||||
_timeline_id: timeline_id,
|
||||
file,
|
||||
len: 0,
|
||||
mutable_tail: Some(BytesMut::zeroed(PAGE_SZ)),
|
||||
rw: page_caching::RW::new(file),
|
||||
})
|
||||
}
|
||||
|
||||
pub(crate) fn len(&self) -> u64 {
|
||||
self.len
|
||||
self.rw.bytes_written()
|
||||
}
|
||||
|
||||
pub(crate) fn id(&self) -> page_cache::FileId {
|
||||
self.page_cache_file_id
|
||||
pub(crate) fn page_cache_file_id(&self) -> page_cache::FileId {
|
||||
self.rw.page_cache_file_id()
|
||||
}
|
||||
|
||||
pub(crate) async fn read_blk(
|
||||
@@ -81,182 +68,30 @@ impl EphemeralFile {
|
||||
blknum: u32,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<BlockLease, io::Error> {
|
||||
let flushed_blknums = 0..self.len / PAGE_SZ as u64;
|
||||
if flushed_blknums.contains(&(blknum as u64)) {
|
||||
let cache = page_cache::get();
|
||||
match cache
|
||||
.read_immutable_buf(self.page_cache_file_id, blknum, ctx)
|
||||
.await
|
||||
.map_err(|e| {
|
||||
std::io::Error::new(
|
||||
std::io::ErrorKind::Other,
|
||||
// order path before error because error is anyhow::Error => might have many contexts
|
||||
format!(
|
||||
"ephemeral file: read immutable page #{}: {}: {:#}",
|
||||
blknum, self.file.path, e,
|
||||
),
|
||||
)
|
||||
})? {
|
||||
page_cache::ReadBufResult::Found(guard) => {
|
||||
return Ok(BlockLease::PageReadGuard(guard))
|
||||
}
|
||||
page_cache::ReadBufResult::NotFound(write_guard) => {
|
||||
let write_guard = self
|
||||
.file
|
||||
.read_exact_at_page(write_guard, blknum as u64 * PAGE_SZ as u64)
|
||||
.await?;
|
||||
let read_guard = write_guard.mark_valid();
|
||||
return Ok(BlockLease::PageReadGuard(read_guard));
|
||||
}
|
||||
};
|
||||
} else {
|
||||
debug_assert_eq!(blknum as u64, self.len / PAGE_SZ as u64);
|
||||
Ok(BlockLease::EphemeralFileMutableTail(
|
||||
self.mutable_tail
|
||||
.as_deref()
|
||||
.expect("we're not doing IO, it must be Some()")
|
||||
.try_into()
|
||||
.expect("we ensure that it's always PAGE_SZ"),
|
||||
))
|
||||
}
|
||||
self.rw.read_blk(blknum, ctx).await
|
||||
}
|
||||
|
||||
pub(crate) async fn write_blob(
|
||||
&mut self,
|
||||
srcbuf: &[u8],
|
||||
ctx: &RequestContext,
|
||||
_ctx: &RequestContext,
|
||||
) -> Result<u64, io::Error> {
|
||||
struct Writer<'a> {
|
||||
ephemeral_file: &'a mut EphemeralFile,
|
||||
/// The block to which the next [`push_bytes`] will write.
|
||||
blknum: u32,
|
||||
/// The offset inside the block identified by [`blknum`] to which [`push_bytes`] will write.
|
||||
off: usize,
|
||||
}
|
||||
impl<'a> Writer<'a> {
|
||||
fn new(ephemeral_file: &'a mut EphemeralFile) -> io::Result<Writer<'a>> {
|
||||
Ok(Writer {
|
||||
blknum: (ephemeral_file.len / PAGE_SZ as u64) as u32,
|
||||
off: (ephemeral_file.len % PAGE_SZ as u64) as usize,
|
||||
ephemeral_file,
|
||||
})
|
||||
}
|
||||
#[inline(always)]
|
||||
async fn push_bytes(
|
||||
&mut self,
|
||||
src: &[u8],
|
||||
ctx: &RequestContext,
|
||||
) -> Result<(), io::Error> {
|
||||
let mut src_remaining = src;
|
||||
while !src_remaining.is_empty() {
|
||||
let dst_remaining = &mut self
|
||||
.ephemeral_file
|
||||
.mutable_tail
|
||||
.as_deref_mut()
|
||||
.expect("IO is not yet ongoing")[self.off..];
|
||||
let n = min(dst_remaining.len(), src_remaining.len());
|
||||
dst_remaining[..n].copy_from_slice(&src_remaining[..n]);
|
||||
self.off += n;
|
||||
src_remaining = &src_remaining[n..];
|
||||
if self.off == PAGE_SZ {
|
||||
let mutable_tail = std::mem::take(&mut self.ephemeral_file.mutable_tail)
|
||||
.expect("IO is not yet ongoing");
|
||||
let (mutable_tail, res) = self
|
||||
.ephemeral_file
|
||||
.file
|
||||
.write_all_at(mutable_tail, self.blknum as u64 * PAGE_SZ as u64)
|
||||
.await;
|
||||
// TODO: If we panic before we can put the mutable_tail back, subsequent calls will fail.
|
||||
// I.e., the IO isn't retryable if we panic.
|
||||
self.ephemeral_file.mutable_tail = Some(mutable_tail);
|
||||
match res {
|
||||
Ok(_) => {
|
||||
// Pre-warm the page cache with what we just wrote.
|
||||
// This isn't necessary for coherency/correctness, but it's how we've always done it.
|
||||
let cache = page_cache::get();
|
||||
match cache
|
||||
.read_immutable_buf(
|
||||
self.ephemeral_file.page_cache_file_id,
|
||||
self.blknum,
|
||||
ctx,
|
||||
)
|
||||
.await
|
||||
{
|
||||
Ok(page_cache::ReadBufResult::Found(_guard)) => {
|
||||
// This function takes &mut self, so, it shouldn't be possible to reach this point.
|
||||
unreachable!("we just wrote blknum {} and this function takes &mut self, so, no concurrent read_blk is possible", self.blknum);
|
||||
}
|
||||
Ok(page_cache::ReadBufResult::NotFound(mut write_guard)) => {
|
||||
let buf: &mut [u8] = write_guard.deref_mut();
|
||||
debug_assert_eq!(buf.len(), PAGE_SZ);
|
||||
buf.copy_from_slice(
|
||||
self.ephemeral_file
|
||||
.mutable_tail
|
||||
.as_deref()
|
||||
.expect("IO is not ongoing"),
|
||||
);
|
||||
let _ = write_guard.mark_valid();
|
||||
// pre-warm successful
|
||||
}
|
||||
Err(e) => {
|
||||
error!("ephemeral_file write_blob failed to get immutable buf to pre-warm page cache: {e:?}");
|
||||
// fail gracefully, it's not the end of the world if we can't pre-warm the cache here
|
||||
}
|
||||
}
|
||||
// Zero the buffer for re-use.
|
||||
// Zeroing is critical for correcntess because the write_blob code below
|
||||
// and similarly read_blk expect zeroed pages.
|
||||
self.ephemeral_file
|
||||
.mutable_tail
|
||||
.as_deref_mut()
|
||||
.expect("IO is not ongoing")
|
||||
.fill(0);
|
||||
// This block is done, move to next one.
|
||||
self.blknum += 1;
|
||||
self.off = 0;
|
||||
}
|
||||
Err(e) => {
|
||||
return Err(std::io::Error::new(
|
||||
ErrorKind::Other,
|
||||
// order error before path because path is long and error is short
|
||||
format!(
|
||||
"ephemeral_file: write_blob: write-back full tail blk #{}: {:#}: {}",
|
||||
self.blknum,
|
||||
e,
|
||||
self.ephemeral_file.file.path,
|
||||
),
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
let pos = self.len;
|
||||
let mut writer = Writer::new(self)?;
|
||||
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];
|
||||
writer.push_bytes(&len_buf, ctx).await?;
|
||||
|
||||
self.rw.write_all_borrowed(&len_buf).await?;
|
||||
} else {
|
||||
let mut len_buf = u32::to_be_bytes(srcbuf.len() as u32);
|
||||
len_buf[0] |= 0x80;
|
||||
writer.push_bytes(&len_buf, ctx).await?;
|
||||
self.rw.write_all_borrowed(&len_buf).await?;
|
||||
}
|
||||
|
||||
// Write the payload
|
||||
writer.push_bytes(srcbuf, ctx).await?;
|
||||
|
||||
if srcbuf.len() < 0x80 {
|
||||
self.len += 1;
|
||||
} else {
|
||||
self.len += 4;
|
||||
}
|
||||
self.len += srcbuf.len() as u64;
|
||||
self.rw.write_all_borrowed(srcbuf).await?;
|
||||
|
||||
Ok(pos)
|
||||
}
|
||||
@@ -271,28 +106,6 @@ pub fn is_ephemeral_file(filename: &str) -> bool {
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for EphemeralFile {
|
||||
fn drop(&mut self) {
|
||||
// There might still be pages in the [`crate::page_cache`] for this file.
|
||||
// We leave them there, [`crate::page_cache::PageCache::find_victim`] will evict them when needed.
|
||||
|
||||
// unlink the file
|
||||
let res = std::fs::remove_file(&self.file.path);
|
||||
if let Err(e) = res {
|
||||
if e.kind() != std::io::ErrorKind::NotFound {
|
||||
// just never log the not found errors, we cannot do anything for them; on detach
|
||||
// the tenant directory is already gone.
|
||||
//
|
||||
// not found files might also be related to https://github.com/neondatabase/neon/issues/2442
|
||||
error!(
|
||||
"could not remove ephemeral file '{}': {}",
|
||||
self.file.path, e
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl BlockReader for EphemeralFile {
|
||||
fn block_cursor(&self) -> super::block_io::BlockCursor<'_> {
|
||||
BlockCursor::new(super::block_io::BlockReaderRef::EphemeralFile(self))
|
||||
|
||||
218
pageserver/src/tenant/ephemeral_file/page_caching.rs
Normal file
218
pageserver/src/tenant/ephemeral_file/page_caching.rs
Normal file
@@ -0,0 +1,218 @@
|
||||
//! Wrapper around [`super::zero_padded_read_write::RW`] that uses the
|
||||
//! [`crate::page_cache`] to serve reads that need to go to the underlying [`VirtualFile`].
|
||||
|
||||
use crate::context::RequestContext;
|
||||
use crate::page_cache::{self, PAGE_SZ};
|
||||
use crate::tenant::block_io::BlockLease;
|
||||
use crate::virtual_file::VirtualFile;
|
||||
|
||||
use once_cell::sync::Lazy;
|
||||
use std::io::{self, ErrorKind};
|
||||
use tokio_epoll_uring::BoundedBuf;
|
||||
use tracing::*;
|
||||
|
||||
use super::zero_padded_read_write;
|
||||
|
||||
/// See module-level comment.
|
||||
pub struct RW {
|
||||
page_cache_file_id: page_cache::FileId,
|
||||
rw: super::zero_padded_read_write::RW<PreWarmingWriter>,
|
||||
}
|
||||
|
||||
impl RW {
|
||||
pub fn new(file: VirtualFile) -> Self {
|
||||
let page_cache_file_id = page_cache::next_file_id();
|
||||
Self {
|
||||
page_cache_file_id,
|
||||
rw: super::zero_padded_read_write::RW::new(PreWarmingWriter::new(
|
||||
page_cache_file_id,
|
||||
file,
|
||||
)),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn page_cache_file_id(&self) -> page_cache::FileId {
|
||||
self.page_cache_file_id
|
||||
}
|
||||
|
||||
pub(crate) async fn write_all_borrowed(&mut self, srcbuf: &[u8]) -> Result<usize, io::Error> {
|
||||
// It doesn't make sense to proactively fill the page cache on the Pageserver write path
|
||||
// because Compute is unlikely to access recently written data.
|
||||
self.rw.write_all_borrowed(srcbuf).await
|
||||
}
|
||||
|
||||
pub(crate) fn bytes_written(&self) -> u64 {
|
||||
self.rw.bytes_written()
|
||||
}
|
||||
|
||||
pub(crate) async fn read_blk(
|
||||
&self,
|
||||
blknum: u32,
|
||||
ctx: &RequestContext,
|
||||
) -> Result<BlockLease, io::Error> {
|
||||
match self.rw.read_blk(blknum).await? {
|
||||
zero_padded_read_write::ReadResult::NeedsReadFromWriter { writer } => {
|
||||
let cache = page_cache::get();
|
||||
match cache
|
||||
.read_immutable_buf(self.page_cache_file_id, blknum, ctx)
|
||||
.await
|
||||
.map_err(|e| {
|
||||
std::io::Error::new(
|
||||
std::io::ErrorKind::Other,
|
||||
// order path before error because error is anyhow::Error => might have many contexts
|
||||
format!(
|
||||
"ephemeral file: read immutable page #{}: {}: {:#}",
|
||||
blknum,
|
||||
self.rw.as_writer().file.path,
|
||||
e,
|
||||
),
|
||||
)
|
||||
})? {
|
||||
page_cache::ReadBufResult::Found(guard) => {
|
||||
return Ok(BlockLease::PageReadGuard(guard))
|
||||
}
|
||||
page_cache::ReadBufResult::NotFound(write_guard) => {
|
||||
let write_guard = writer
|
||||
.file
|
||||
.read_exact_at_page(write_guard, blknum as u64 * PAGE_SZ as u64)
|
||||
.await?;
|
||||
let read_guard = write_guard.mark_valid();
|
||||
return Ok(BlockLease::PageReadGuard(read_guard));
|
||||
}
|
||||
}
|
||||
}
|
||||
zero_padded_read_write::ReadResult::ServedFromZeroPaddedMutableTail { buffer } => {
|
||||
Ok(BlockLease::EphemeralFileMutableTail(buffer))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for RW {
|
||||
fn drop(&mut self) {
|
||||
// There might still be pages in the [`crate::page_cache`] for this file.
|
||||
// We leave them there, [`crate::page_cache::PageCache::find_victim`] will evict them when needed.
|
||||
|
||||
// unlink the file
|
||||
let res = std::fs::remove_file(&self.rw.as_writer().file.path);
|
||||
if let Err(e) = res {
|
||||
if e.kind() != std::io::ErrorKind::NotFound {
|
||||
// just never log the not found errors, we cannot do anything for them; on detach
|
||||
// the tenant directory is already gone.
|
||||
//
|
||||
// not found files might also be related to https://github.com/neondatabase/neon/issues/2442
|
||||
error!(
|
||||
"could not remove ephemeral file '{}': {}",
|
||||
self.rw.as_writer().file.path,
|
||||
e
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct PreWarmingWriter {
|
||||
nwritten_blocks: u32,
|
||||
page_cache_file_id: page_cache::FileId,
|
||||
file: VirtualFile,
|
||||
}
|
||||
|
||||
impl PreWarmingWriter {
|
||||
fn new(page_cache_file_id: page_cache::FileId, file: VirtualFile) -> Self {
|
||||
Self {
|
||||
nwritten_blocks: 0,
|
||||
page_cache_file_id,
|
||||
file,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl crate::virtual_file::owned_buffers_io::write::OwnedAsyncWriter for PreWarmingWriter {
|
||||
async fn write_all<
|
||||
B: tokio_epoll_uring::BoundedBuf<Buf = Buf>,
|
||||
Buf: tokio_epoll_uring::IoBuf + Send,
|
||||
>(
|
||||
&mut self,
|
||||
buf: B,
|
||||
) -> std::io::Result<(usize, B::Buf)> {
|
||||
let buf = buf.slice(..);
|
||||
let saved_bounds = buf.bounds(); // save for reconstructing the Slice from iobuf after the IO is done
|
||||
let check_bounds_stuff_works = if cfg!(test) && cfg!(debug_assertions) {
|
||||
Some(buf.to_vec())
|
||||
} else {
|
||||
None
|
||||
};
|
||||
let buflen = buf.len();
|
||||
assert_eq!(
|
||||
buflen % PAGE_SZ,
|
||||
0,
|
||||
"{buflen} ; we know TAIL_SZ is a PAGE_SZ multiple, and write_buffered_borrowed is used"
|
||||
);
|
||||
|
||||
// Do the IO.
|
||||
let iobuf = match self.file.write_all(buf).await {
|
||||
(iobuf, Ok(nwritten)) => {
|
||||
assert_eq!(nwritten, buflen);
|
||||
iobuf
|
||||
}
|
||||
(_, Err(e)) => {
|
||||
return Err(std::io::Error::new(
|
||||
ErrorKind::Other,
|
||||
// order error before path because path is long and error is short
|
||||
format!(
|
||||
"ephemeral_file: write_blob: write-back tail self.nwritten_blocks={}, buflen={}, {:#}: {}",
|
||||
self.nwritten_blocks, buflen, e, self.file.path,
|
||||
),
|
||||
));
|
||||
}
|
||||
};
|
||||
|
||||
// Reconstruct the Slice (the write path consumed the Slice and returned us the underlying IoBuf)
|
||||
let buf = tokio_epoll_uring::Slice::from_buf_bounds(iobuf, saved_bounds);
|
||||
if let Some(check_bounds_stuff_works) = check_bounds_stuff_works {
|
||||
assert_eq!(&check_bounds_stuff_works, &*buf);
|
||||
}
|
||||
|
||||
// Pre-warm page cache with the contents.
|
||||
// At least in isolated bulk ingest benchmarks (test_bulk_insert.py), the pre-warming
|
||||
// benefits the code that writes InMemoryLayer=>L0 layers.
|
||||
let nblocks = buflen / PAGE_SZ;
|
||||
let nblocks32 = u32::try_from(nblocks).unwrap();
|
||||
let cache = page_cache::get();
|
||||
static CTX: Lazy<RequestContext> = Lazy::new(|| {
|
||||
RequestContext::new(
|
||||
crate::task_mgr::TaskKind::EphemeralFilePreWarmPageCache,
|
||||
crate::context::DownloadBehavior::Error,
|
||||
)
|
||||
});
|
||||
for blknum_in_buffer in 0..nblocks {
|
||||
let blk_in_buffer = &buf[blknum_in_buffer * PAGE_SZ..(blknum_in_buffer + 1) * PAGE_SZ];
|
||||
let blknum = self
|
||||
.nwritten_blocks
|
||||
.checked_add(blknum_in_buffer as u32)
|
||||
.unwrap();
|
||||
match cache
|
||||
.read_immutable_buf(self.page_cache_file_id, blknum, &CTX)
|
||||
.await
|
||||
{
|
||||
Err(e) => {
|
||||
error!("ephemeral_file write_blob failed to get immutable buf to pre-warm page cache: {e:?}");
|
||||
// fail gracefully, it's not the end of the world if we can't pre-warm the cache here
|
||||
}
|
||||
Ok(v) => match v {
|
||||
page_cache::ReadBufResult::Found(_guard) => {
|
||||
// This function takes &mut self, so, it shouldn't be possible to reach this point.
|
||||
unreachable!("we just wrote block {blknum} to the VirtualFile, which is owned by Self, \
|
||||
and this function takes &mut self, so, no concurrent read_blk is possible");
|
||||
}
|
||||
page_cache::ReadBufResult::NotFound(mut write_guard) => {
|
||||
write_guard.copy_from_slice(blk_in_buffer);
|
||||
let _ = write_guard.mark_valid();
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
self.nwritten_blocks = self.nwritten_blocks.checked_add(nblocks32).unwrap();
|
||||
Ok((buflen, buf.into_inner()))
|
||||
}
|
||||
}
|
||||
125
pageserver/src/tenant/ephemeral_file/zero_padded_read_write.rs
Normal file
125
pageserver/src/tenant/ephemeral_file/zero_padded_read_write.rs
Normal file
@@ -0,0 +1,125 @@
|
||||
//! The heart of how [`super::EphemeralFile`] does its reads and writes.
|
||||
//!
|
||||
//! # Writes
|
||||
//!
|
||||
//! [`super::EphemeralFile`] writes small, borrowed buffers using [`RW::write_all_borrowed`].
|
||||
//! The [`RW`] batches these into [`TAIL_SZ`] bigger writes, using [`owned_buffers_io::write::BufferedWriter`].
|
||||
//!
|
||||
//! # Reads
|
||||
//!
|
||||
//! [`super::EphemeralFile`] always reads full [`PAGE_SZ`]ed blocks using [`RW::read_blk`].
|
||||
//!
|
||||
//! The [`RW`] serves these reads either from the buffered writer's in-memory buffer
|
||||
//! or redirects the caller to read from the underlying [`OwnedAsyncWriter`]
|
||||
//! if the read is for the prefix that has already been flushed.
|
||||
//!
|
||||
//! # Current Usage
|
||||
//!
|
||||
//! The current user of this module is [`super::page_caching::RW`].
|
||||
|
||||
mod zero_padded;
|
||||
|
||||
use crate::{
|
||||
page_cache::PAGE_SZ,
|
||||
virtual_file::owned_buffers_io::{
|
||||
self,
|
||||
write::{Buffer, OwnedAsyncWriter},
|
||||
},
|
||||
};
|
||||
|
||||
const TAIL_SZ: usize = PAGE_SZ;
|
||||
|
||||
/// See module-level comment.
|
||||
pub struct RW<W: OwnedAsyncWriter> {
|
||||
buffered_writer: owned_buffers_io::write::BufferedWriter<
|
||||
zero_padded::Buffer<TAIL_SZ>,
|
||||
owned_buffers_io::util::size_tracking_writer::Writer<W>,
|
||||
>,
|
||||
}
|
||||
|
||||
pub enum ReadResult<'a, W> {
|
||||
NeedsReadFromWriter { writer: &'a W },
|
||||
ServedFromZeroPaddedMutableTail { buffer: &'a [u8; PAGE_SZ] },
|
||||
}
|
||||
|
||||
impl<W> RW<W>
|
||||
where
|
||||
W: OwnedAsyncWriter,
|
||||
{
|
||||
pub fn new(writer: W) -> Self {
|
||||
let bytes_flushed_tracker =
|
||||
owned_buffers_io::util::size_tracking_writer::Writer::new(writer);
|
||||
let buffered_writer = owned_buffers_io::write::BufferedWriter::new(
|
||||
bytes_flushed_tracker,
|
||||
zero_padded::Buffer::default(),
|
||||
);
|
||||
Self { buffered_writer }
|
||||
}
|
||||
|
||||
pub(crate) fn as_writer(&self) -> &W {
|
||||
self.buffered_writer.as_inner().as_inner()
|
||||
}
|
||||
|
||||
pub async fn write_all_borrowed(&mut self, buf: &[u8]) -> std::io::Result<usize> {
|
||||
self.buffered_writer.write_buffered_borrowed(buf).await
|
||||
}
|
||||
|
||||
pub fn bytes_written(&self) -> u64 {
|
||||
let flushed_offset = self.buffered_writer.as_inner().bytes_written();
|
||||
let buffer: &zero_padded::Buffer<TAIL_SZ> = self.buffered_writer.inspect_buffer();
|
||||
flushed_offset + u64::try_from(buffer.pending()).unwrap()
|
||||
}
|
||||
|
||||
pub(crate) async fn read_blk(&self, blknum: u32) -> Result<ReadResult<'_, W>, std::io::Error> {
|
||||
let flushed_offset = self.buffered_writer.as_inner().bytes_written();
|
||||
let buffer: &zero_padded::Buffer<TAIL_SZ> = self.buffered_writer.inspect_buffer();
|
||||
let buffered_offset = flushed_offset + u64::try_from(buffer.pending()).unwrap();
|
||||
let read_offset = (blknum as u64) * (PAGE_SZ as u64);
|
||||
|
||||
// The trailing page ("block") might only be partially filled,
|
||||
// yet the blob_io code relies on us to return a full PAGE_SZed slice anyway.
|
||||
// Moreover, it has to be zero-padded, because when we still had
|
||||
// a write-back page cache, it provided pre-zeroed pages, and blob_io came to rely on it.
|
||||
// DeltaLayer probably has the same issue, not sure why it needs no special treatment.
|
||||
// => check here that the read doesn't go beyond this potentially trailing
|
||||
// => the zero-padding is done in the `else` branch below
|
||||
let blocks_written = if buffered_offset % (PAGE_SZ as u64) == 0 {
|
||||
buffered_offset / (PAGE_SZ as u64)
|
||||
} else {
|
||||
(buffered_offset / (PAGE_SZ as u64)) + 1
|
||||
};
|
||||
if (blknum as u64) >= blocks_written {
|
||||
return Err(std::io::Error::new(std::io::ErrorKind::Other, anyhow::anyhow!("read past end of ephemeral_file: read=0x{read_offset:x} buffered=0x{buffered_offset:x} flushed=0x{flushed_offset}")));
|
||||
}
|
||||
|
||||
// assertions for the `if-else` below
|
||||
assert_eq!(
|
||||
flushed_offset % (TAIL_SZ as u64), 0,
|
||||
"we only use write_buffered_borrowed to write to the buffered writer, so it's guaranteed that flushes happen buffer.cap()-sized chunks"
|
||||
);
|
||||
assert_eq!(
|
||||
flushed_offset % (PAGE_SZ as u64),
|
||||
0,
|
||||
"the logic below can't handle if the page is spread across the flushed part and the buffer"
|
||||
);
|
||||
|
||||
if read_offset < flushed_offset {
|
||||
assert!(read_offset + (PAGE_SZ as u64) <= flushed_offset);
|
||||
Ok(ReadResult::NeedsReadFromWriter {
|
||||
writer: self.as_writer(),
|
||||
})
|
||||
} else {
|
||||
let read_offset_in_buffer = read_offset
|
||||
.checked_sub(flushed_offset)
|
||||
.expect("would have taken `if` branch instead of this one");
|
||||
let read_offset_in_buffer = usize::try_from(read_offset_in_buffer).unwrap();
|
||||
let zero_padded_slice = buffer.as_zero_padded_slice();
|
||||
let page = &zero_padded_slice[read_offset_in_buffer..(read_offset_in_buffer + PAGE_SZ)];
|
||||
Ok(ReadResult::ServedFromZeroPaddedMutableTail {
|
||||
buffer: page
|
||||
.try_into()
|
||||
.expect("the slice above got it as page-size slice"),
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,108 @@
|
||||
//! A [`crate::virtual_file::owned_buffers_io::write::Buffer`] whose
|
||||
//! unwritten range is guaranteed to be zero-initialized.
|
||||
//! This is used by [`crate::tenant::ephemeral_file::zero_padded_read_write::RW::read_blk`]
|
||||
//! to serve page-sized reads of the trailing page when the trailing page has only been partially filled.
|
||||
|
||||
use std::mem::MaybeUninit;
|
||||
|
||||
/// See module-level comment.
|
||||
pub struct Buffer<const N: usize> {
|
||||
allocation: Box<[u8; N]>,
|
||||
written: usize,
|
||||
}
|
||||
|
||||
impl<const N: usize> Default for Buffer<N> {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
allocation: Box::new(
|
||||
// SAFETY: zeroed memory is a valid [u8; N]
|
||||
unsafe { MaybeUninit::zeroed().assume_init() },
|
||||
),
|
||||
written: 0,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<const N: usize> Buffer<N> {
|
||||
#[inline(always)]
|
||||
fn invariants(&self) {
|
||||
// don't check by default, unoptimized is too expensive even for debug mode
|
||||
if false {
|
||||
debug_assert!(self.written <= N, "{}", self.written);
|
||||
debug_assert!(self.allocation[self.written..N].iter().all(|v| *v == 0));
|
||||
}
|
||||
}
|
||||
|
||||
pub fn as_zero_padded_slice(&self) -> &[u8; N] {
|
||||
&self.allocation
|
||||
}
|
||||
}
|
||||
|
||||
impl<const N: usize> crate::virtual_file::owned_buffers_io::write::Buffer for Buffer<N> {
|
||||
type IoBuf = Self;
|
||||
|
||||
fn cap(&self) -> usize {
|
||||
self.allocation.len()
|
||||
}
|
||||
|
||||
fn extend_from_slice(&mut self, other: &[u8]) {
|
||||
self.invariants();
|
||||
let remaining = self.allocation.len() - self.written;
|
||||
if other.len() > remaining {
|
||||
panic!("calling extend_from_slice() with insufficient remaining capacity");
|
||||
}
|
||||
self.allocation[self.written..(self.written + other.len())].copy_from_slice(other);
|
||||
self.written += other.len();
|
||||
self.invariants();
|
||||
}
|
||||
|
||||
fn pending(&self) -> usize {
|
||||
self.written
|
||||
}
|
||||
|
||||
fn flush(self) -> tokio_epoll_uring::Slice<Self> {
|
||||
self.invariants();
|
||||
let written = self.written;
|
||||
tokio_epoll_uring::BoundedBuf::slice(self, 0..written)
|
||||
}
|
||||
|
||||
fn reuse_after_flush(iobuf: Self::IoBuf) -> Self {
|
||||
let Self {
|
||||
mut allocation,
|
||||
written,
|
||||
} = iobuf;
|
||||
allocation[0..written].fill(0);
|
||||
let new = Self {
|
||||
allocation,
|
||||
written: 0,
|
||||
};
|
||||
new.invariants();
|
||||
new
|
||||
}
|
||||
}
|
||||
|
||||
/// We have this trait impl so that the `flush` method in the `Buffer` impl above can produce a
|
||||
/// [`tokio_epoll_uring::BoundedBuf::slice`] of the [`Self::written`] range of the data.
|
||||
///
|
||||
/// Remember that bytes_init is generally _not_ a tracker of the amount
|
||||
/// of valid data in the io buffer; we use `Slice` for that.
|
||||
/// The `IoBuf` is _only_ for keeping track of uninitialized memory, a bit like MaybeUninit.
|
||||
///
|
||||
/// SAFETY:
|
||||
///
|
||||
/// The [`Self::allocation`] is stable becauses boxes are stable.
|
||||
/// The memory is zero-initialized, so, bytes_init is always N.
|
||||
unsafe impl<const N: usize> tokio_epoll_uring::IoBuf for Buffer<N> {
|
||||
fn stable_ptr(&self) -> *const u8 {
|
||||
self.allocation.as_ptr()
|
||||
}
|
||||
|
||||
fn bytes_init(&self) -> usize {
|
||||
// Yes, N, not self.written; Read the full comment of this impl block!
|
||||
N
|
||||
}
|
||||
|
||||
fn bytes_total(&self) -> usize {
|
||||
N
|
||||
}
|
||||
}
|
||||
@@ -7,7 +7,6 @@ use std::collections::HashSet;
|
||||
use std::future::Future;
|
||||
|
||||
use anyhow::{anyhow, Context};
|
||||
use bytes::BytesMut;
|
||||
use camino::{Utf8Path, Utf8PathBuf};
|
||||
use pageserver_api::shard::TenantShardId;
|
||||
use tokio::fs::{self, File, OpenOptions};
|
||||
|
||||
@@ -482,7 +482,7 @@ impl InMemoryLayer {
|
||||
trace!("initializing new empty InMemoryLayer for writing on timeline {timeline_id} at {start_lsn}");
|
||||
|
||||
let file = EphemeralFile::create(conf, tenant_shard_id, timeline_id).await?;
|
||||
let key = InMemoryLayerFileId(file.id());
|
||||
let key = InMemoryLayerFileId(file.page_cache_file_id());
|
||||
|
||||
Ok(InMemoryLayer {
|
||||
file_id: key,
|
||||
|
||||
@@ -37,7 +37,6 @@ pub(crate) use io_engine::IoEngineKind;
|
||||
pub(crate) use metadata::Metadata;
|
||||
pub(crate) use open_options::*;
|
||||
|
||||
#[cfg_attr(not(target_os = "linux"), allow(dead_code))]
|
||||
pub(crate) mod owned_buffers_io {
|
||||
//! Abstractions for IO with owned buffers.
|
||||
//!
|
||||
|
||||
@@ -14,6 +14,14 @@ impl<W> Writer<W> {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn bytes_written(&self) -> u64 {
|
||||
self.bytes_amount
|
||||
}
|
||||
|
||||
pub fn as_inner(&self) -> &W {
|
||||
&self.dst
|
||||
}
|
||||
|
||||
/// Returns the wrapped `VirtualFile` object as well as the number
|
||||
/// of bytes that were written to it through this object.
|
||||
pub fn into_inner(self) -> (u64, W) {
|
||||
|
||||
@@ -47,6 +47,15 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
pub fn as_inner(&self) -> &W {
|
||||
&self.writer
|
||||
}
|
||||
|
||||
/// Panics if used after any of the write paths returned an error
|
||||
pub fn inspect_buffer(&self) -> &B {
|
||||
self.buf()
|
||||
}
|
||||
|
||||
pub async fn flush_and_into_inner(mut self) -> std::io::Result<W> {
|
||||
self.flush().await?;
|
||||
let Self { buf, writer } = self;
|
||||
@@ -100,6 +109,28 @@ where
|
||||
Ok((chunk_len, chunk.into_inner()))
|
||||
}
|
||||
|
||||
/// Strictly less performant variant of [`Self::write_buffered`] that allows writing borrowed data.
|
||||
///
|
||||
/// It is less performant because we always have to copy the borrowed data into the internal buffer
|
||||
/// before we can do the IO. The [`Self::write_buffered`] can avoid this, which is more performant
|
||||
/// for large writes.
|
||||
pub async fn write_buffered_borrowed(&mut self, mut chunk: &[u8]) -> std::io::Result<usize> {
|
||||
let chunk_len = chunk.len();
|
||||
while !chunk.is_empty() {
|
||||
let buf = self.buf.as_mut().expect("must not use after an error");
|
||||
let need = buf.cap() - buf.pending();
|
||||
let have = chunk.len();
|
||||
let n = std::cmp::min(need, have);
|
||||
buf.extend_from_slice(&chunk[..n]);
|
||||
chunk = &chunk[n..];
|
||||
if buf.pending() >= buf.cap() {
|
||||
assert_eq!(buf.pending(), buf.cap());
|
||||
self.flush().await?;
|
||||
}
|
||||
}
|
||||
Ok(chunk_len)
|
||||
}
|
||||
|
||||
async fn flush(&mut self) -> std::io::Result<()> {
|
||||
let buf = self.buf.take().expect("must not use after an error");
|
||||
let buf_len = buf.pending();
|
||||
@@ -266,4 +297,31 @@ mod tests {
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_write_all_borrowed_always_goes_through_buffer() -> std::io::Result<()> {
|
||||
let recorder = RecorderWriter::default();
|
||||
let mut writer = BufferedWriter::new(recorder, BytesMut::with_capacity(2));
|
||||
|
||||
writer.write_buffered_borrowed(b"abc").await?;
|
||||
writer.write_buffered_borrowed(b"d").await?;
|
||||
writer.write_buffered_borrowed(b"e").await?;
|
||||
writer.write_buffered_borrowed(b"fg").await?;
|
||||
writer.write_buffered_borrowed(b"hi").await?;
|
||||
writer.write_buffered_borrowed(b"j").await?;
|
||||
writer.write_buffered_borrowed(b"klmno").await?;
|
||||
|
||||
let recorder = writer.flush_and_into_inner().await?;
|
||||
assert_eq!(
|
||||
recorder.writes,
|
||||
{
|
||||
let expect: &[&[u8]] = &[b"ab", b"cd", b"ef", b"gh", b"ij", b"kl", b"mn", b"o"];
|
||||
expect
|
||||
}
|
||||
.iter()
|
||||
.map(|v| v[..].to_vec())
|
||||
.collect::<Vec<_>>()
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user