get rid of io_buffer_alignment config (always 512)

Signed-off-by: Yuchen Liang <yuchen@neon.tech>
This commit is contained in:
Yuchen Liang
2024-10-07 12:16:11 -04:00
parent bc13310e56
commit a04cfd754b
15 changed files with 54 additions and 201 deletions

View File

@@ -105,7 +105,6 @@ pub struct ConfigToml {
pub ephemeral_bytes_per_memory_kb: usize,
pub l0_flush: Option<crate::models::L0FlushConfig>,
pub virtual_file_io_mode: Option<crate::models::virtual_file::IoMode>,
pub io_buffer_alignment: usize,
}
#[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
@@ -389,9 +388,6 @@ impl Default for ConfigToml {
ephemeral_bytes_per_memory_kb: (DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB),
l0_flush: None,
virtual_file_io_mode: None,
io_buffer_alignment: DEFAULT_IO_BUFFER_ALIGNMENT,
tenant_config: TenantConfigToml::default(),
}
}

View File

@@ -164,11 +164,7 @@ fn criterion_benchmark(c: &mut Criterion) {
let conf: &'static PageServerConf = Box::leak(Box::new(
pageserver::config::PageServerConf::dummy_conf(temp_dir.path().to_path_buf()),
));
virtual_file::init(
16384,
virtual_file::io_engine_for_bench(),
pageserver_api::config::defaults::DEFAULT_IO_BUFFER_ALIGNMENT,
);
virtual_file::init(16384, virtual_file::io_engine_for_bench());
page_cache::init(conf.page_cache_size);
{

View File

@@ -540,16 +540,6 @@ impl Client {
.map_err(Error::ReceiveBody)
}
/// Configs io buffer alignment at runtime.
pub async fn put_io_alignment(&self, align: usize) -> Result<()> {
let uri = format!("{}/v1/io_alignment", self.mgmt_api_endpoint);
self.request(Method::PUT, uri, align)
.await?
.json()
.await
.map_err(Error::ReceiveBody)
}
/// Configs io mode at runtime.
pub async fn put_io_mode(
&self,

View File

@@ -152,11 +152,7 @@ pub(crate) async fn main(cmd: &AnalyzeLayerMapCmd) -> Result<()> {
let ctx = RequestContext::new(TaskKind::DebugTool, DownloadBehavior::Error);
// Initialize virtual_file (file desriptor cache) and page cache which are needed to access layer persistent B-Tree.
pageserver::virtual_file::init(
10,
virtual_file::api::IoEngineKind::StdFs,
pageserver_api::config::defaults::DEFAULT_IO_BUFFER_ALIGNMENT,
);
pageserver::virtual_file::init(10, virtual_file::api::IoEngineKind::StdFs);
pageserver::page_cache::init(100);
let mut total_delta_layers = 0usize;

View File

@@ -59,7 +59,7 @@ pub(crate) enum LayerCmd {
async fn read_delta_file(path: impl AsRef<Path>, ctx: &RequestContext) -> Result<()> {
let path = Utf8Path::from_path(path.as_ref()).expect("non-Unicode path");
virtual_file::init(10, virtual_file::api::IoEngineKind::StdFs, 1);
virtual_file::init(10, virtual_file::api::IoEngineKind::StdFs);
page_cache::init(100);
let file = VirtualFile::open(path, ctx).await?;
let file_id = page_cache::next_file_id();
@@ -190,11 +190,7 @@ pub(crate) async fn main(cmd: &LayerCmd) -> Result<()> {
new_tenant_id,
new_timeline_id,
} => {
pageserver::virtual_file::init(
10,
virtual_file::api::IoEngineKind::StdFs,
pageserver_api::config::defaults::DEFAULT_IO_BUFFER_ALIGNMENT,
);
pageserver::virtual_file::init(10, virtual_file::api::IoEngineKind::StdFs);
pageserver::page_cache::init(100);
let ctx = RequestContext::new(TaskKind::DebugTool, DownloadBehavior::Error);

View File

@@ -26,7 +26,7 @@ use pageserver::{
tenant::{dump_layerfile_from_path, metadata::TimelineMetadata},
virtual_file,
};
use pageserver_api::{config::defaults::DEFAULT_IO_BUFFER_ALIGNMENT, shard::TenantShardId};
use pageserver_api::shard::TenantShardId;
use postgres_ffi::ControlFileData;
use remote_storage::{RemotePath, RemoteStorageConfig};
use tokio_util::sync::CancellationToken;
@@ -205,11 +205,7 @@ fn read_pg_control_file(control_file_path: &Utf8Path) -> anyhow::Result<()> {
async fn print_layerfile(path: &Utf8Path) -> anyhow::Result<()> {
// Basic initialization of things that don't change after startup
virtual_file::init(
10,
virtual_file::api::IoEngineKind::StdFs,
DEFAULT_IO_BUFFER_ALIGNMENT,
);
virtual_file::init(10, virtual_file::api::IoEngineKind::StdFs);
page_cache::init(100);
let ctx = RequestContext::new(TaskKind::DebugTool, DownloadBehavior::Error);
dump_layerfile_from_path(path, true, &ctx).await

View File

@@ -59,10 +59,6 @@ pub(crate) struct Args {
#[clap(long)]
set_io_engine: Option<pageserver_api::models::virtual_file::IoEngineKind>,
/// Before starting the benchmark, live-reconfigure the pageserver to use specified alignment for io buffers.
#[clap(long)]
set_io_alignment: Option<usize>,
/// Before starting the benchmark, live-reconfigure the pageserver to use specified io mode (buffered vs. direct).
#[clap(long)]
set_io_mode: Option<pageserver_api::models::virtual_file::IoMode>,
@@ -133,10 +129,6 @@ async fn main_impl(
mgmt_api_client.put_io_engine(engine_str).await?;
}
if let Some(align) = args.set_io_alignment {
mgmt_api_client.put_io_alignment(align).await?;
}
if let Some(mode) = &args.set_io_mode {
mgmt_api_client.put_io_mode(mode).await?;
}

View File

@@ -125,8 +125,7 @@ fn main() -> anyhow::Result<()> {
// after setting up logging, log the effective IO engine choice and read path implementations
info!(?conf.virtual_file_io_engine, "starting with virtual_file IO engine");
info!(?conf.virtual_file_io_mode, "starting with virtual_file Direct IO settings");
info!(?conf.io_buffer_alignment, "starting with setting for IO buffer alignment");
info!(?conf.virtual_file_io_mode, "starting with virtual_file IO mode");
// The tenants directory contains all the pageserver local disk state.
// Create if not exists and make sure all the contents are durable before proceeding.
@@ -168,11 +167,7 @@ fn main() -> anyhow::Result<()> {
let scenario = failpoint_support::init();
// Basic initialization of things that don't change after startup
virtual_file::init(
conf.max_file_descriptors,
conf.virtual_file_io_engine,
conf.io_buffer_alignment,
);
virtual_file::init(conf.max_file_descriptors, conf.virtual_file_io_engine);
page_cache::init(conf.page_cache_size);
start_pageserver(launch_ts, conf).context("Failed to start pageserver")?;

View File

@@ -175,8 +175,6 @@ pub struct PageServerConf {
/// Direct IO settings
pub virtual_file_io_mode: virtual_file::IoMode,
pub io_buffer_alignment: usize,
}
/// Token for authentication to safekeepers
@@ -329,7 +327,6 @@ impl PageServerConf {
concurrent_tenant_warmup,
concurrent_tenant_size_logical_size_queries,
virtual_file_io_engine,
io_buffer_alignment,
tenant_config,
} = config_toml;
@@ -368,7 +365,6 @@ impl PageServerConf {
max_vectored_read_bytes,
image_compression,
ephemeral_bytes_per_memory_kb,
io_buffer_alignment,
// ------------------------------------------------------------
// fields that require additional validation or custom handling

View File

@@ -2380,20 +2380,6 @@ async fn put_io_engine_handler(
json_response(StatusCode::OK, ())
}
async fn put_io_alignment_handler(
mut r: Request<Body>,
_cancel: CancellationToken,
) -> Result<Response<Body>, ApiError> {
check_permission(&r, None)?;
let align: usize = json_request(&mut r).await?;
crate::virtual_file::set_io_buffer_alignment(align).map_err(|align| {
ApiError::PreconditionFailed(
format!("Requested io alignment ({align}) is not a power of two").into(),
)
})?;
json_response(StatusCode::OK, ())
}
async fn put_io_mode_handler(
mut r: Request<Body>,
_cancel: CancellationToken,
@@ -3091,9 +3077,6 @@ pub fn make_router(
|r| api_handler(r, timeline_collect_keyspace),
)
.put("/v1/io_engine", |r| api_handler(r, put_io_engine_handler))
.put("/v1/io_alignment", |r| {
api_handler(r, put_io_alignment_handler)
})
.put("/v1/io_mode", |r| api_handler(r, put_io_mode_handler))
.put(
"/v1/tenant/:tenant_shard_id/timeline/:timeline_id/force_aux_policy_switch",

View File

@@ -1198,7 +1198,6 @@ impl DeltaLayerInner {
let mut prev: Option<(Key, Lsn, BlobRef)> = None;
let mut read_builder: Option<ChunkedVectoredReadBuilder> = None;
let align = virtual_file::get_io_buffer_alignment();
let max_read_size = self
.max_vectored_read_bytes
@@ -1247,7 +1246,6 @@ impl DeltaLayerInner {
offsets.end.pos(),
meta,
max_read_size,
align,
))
}
} else {

View File

@@ -194,8 +194,6 @@ pub(crate) struct ChunkedVectoredReadBuilder {
/// Start offset and metadata for each blob in this read
blobs_at: VecMap<u64, BlobMeta>,
max_read_size: Option<usize>,
/// Chunk size reads are coalesced into.
chunk_size: usize,
}
/// Computes x / d rounded up.
@@ -204,6 +202,7 @@ fn div_round_up(x: usize, d: usize) -> usize {
}
impl ChunkedVectoredReadBuilder {
const CHUNK_SIZE: usize = virtual_file::get_io_buffer_alignment();
/// Start building a new vectored read.
///
/// Note that by design, this does not check against reading more than `max_read_size` to
@@ -214,21 +213,19 @@ impl ChunkedVectoredReadBuilder {
end_offset: u64,
meta: BlobMeta,
max_read_size: Option<usize>,
chunk_size: usize,
) -> Self {
let mut blobs_at = VecMap::default();
blobs_at
.append(start_offset, meta)
.expect("First insertion always succeeds");
let start_blk_no = start_offset as usize / chunk_size;
let end_blk_no = div_round_up(end_offset as usize, chunk_size);
let start_blk_no = start_offset as usize / Self::CHUNK_SIZE;
let end_blk_no = div_round_up(end_offset as usize, Self::CHUNK_SIZE);
Self {
start_blk_no,
end_blk_no,
blobs_at,
max_read_size,
chunk_size,
}
}
@@ -237,18 +234,12 @@ impl ChunkedVectoredReadBuilder {
end_offset: u64,
meta: BlobMeta,
max_read_size: usize,
align: usize,
) -> Self {
Self::new_impl(start_offset, end_offset, meta, Some(max_read_size), align)
Self::new_impl(start_offset, end_offset, meta, Some(max_read_size))
}
pub(crate) fn new_streaming(
start_offset: u64,
end_offset: u64,
meta: BlobMeta,
align: usize,
) -> Self {
Self::new_impl(start_offset, end_offset, meta, None, align)
pub(crate) fn new_streaming(start_offset: u64, end_offset: u64, meta: BlobMeta) -> Self {
Self::new_impl(start_offset, end_offset, meta, None)
}
/// Attempts to extend the current read with a new blob if the new blob resides in the same or the immediate next chunk.
@@ -256,12 +247,12 @@ impl ChunkedVectoredReadBuilder {
/// The resulting size also must be below the max read size.
pub(crate) fn extend(&mut self, start: u64, end: u64, meta: BlobMeta) -> VectoredReadExtended {
tracing::trace!(start, end, "trying to extend");
let start_blk_no = start as usize / self.chunk_size;
let end_blk_no = div_round_up(end as usize, self.chunk_size);
let start_blk_no = start as usize / Self::CHUNK_SIZE;
let end_blk_no = div_round_up(end as usize, Self::CHUNK_SIZE);
let not_limited_by_max_read_size = {
if let Some(max_read_size) = self.max_read_size {
let coalesced_size = (end_blk_no - self.start_blk_no) * self.chunk_size;
let coalesced_size = (end_blk_no - self.start_blk_no) * Self::CHUNK_SIZE;
coalesced_size <= max_read_size
} else {
true
@@ -292,12 +283,12 @@ impl ChunkedVectoredReadBuilder {
}
pub(crate) fn size(&self) -> usize {
(self.end_blk_no - self.start_blk_no) * self.chunk_size
(self.end_blk_no - self.start_blk_no) * Self::CHUNK_SIZE
}
pub(crate) fn build(self) -> VectoredRead {
let start = (self.start_blk_no * self.chunk_size) as u64;
let end = (self.end_blk_no * self.chunk_size) as u64;
let start = (self.start_blk_no * Self::CHUNK_SIZE) as u64;
let end = (self.end_blk_no * Self::CHUNK_SIZE) as u64;
VectoredRead {
start,
end,
@@ -328,18 +319,14 @@ pub struct VectoredReadPlanner {
prev: Option<(Key, Lsn, u64, BlobFlag)>,
max_read_size: usize,
align: usize,
}
impl VectoredReadPlanner {
pub fn new(max_read_size: usize) -> Self {
let align = virtual_file::get_io_buffer_alignment();
Self {
blobs: BTreeMap::new(),
prev: None,
max_read_size,
align,
}
}
@@ -418,7 +405,6 @@ impl VectoredReadPlanner {
end_offset,
BlobMeta { key, lsn },
self.max_read_size,
self.align,
);
let prev_read_builder = current_read_builder.replace(next_read_builder);
@@ -472,13 +458,13 @@ impl<'a> VectoredBlobReader<'a> {
);
if cfg!(debug_assertions) {
let align = virtual_file::get_io_buffer_alignment() as u64;
const ALIGN: u64 = virtual_file::get_io_buffer_alignment() as u64;
debug_assert_eq!(
read.start % align,
read.start % ALIGN,
0,
"Read start at {} does not satisfy the required io buffer alignment ({} bytes)",
read.start,
align
ALIGN
);
}
@@ -553,22 +539,18 @@ pub struct StreamingVectoredReadPlanner {
max_cnt: usize,
/// Size of the current batch
cnt: usize,
align: usize,
}
impl StreamingVectoredReadPlanner {
pub fn new(max_read_size: u64, max_cnt: usize) -> Self {
assert!(max_cnt > 0);
assert!(max_read_size > 0);
let align = virtual_file::get_io_buffer_alignment();
Self {
read_builder: None,
prev: None,
max_cnt,
max_read_size,
cnt: 0,
align,
}
}
@@ -621,7 +603,6 @@ impl StreamingVectoredReadPlanner {
start_offset,
end_offset,
BlobMeta { key, lsn },
self.align,
))
};
}
@@ -656,9 +637,9 @@ mod tests {
use super::*;
fn validate_read(read: &VectoredRead, offset_range: &[(Key, Lsn, u64, BlobFlag)]) {
let align = virtual_file::get_io_buffer_alignment() as u64;
assert_eq!(read.start % align, 0);
assert_eq!(read.start / align, offset_range.first().unwrap().2 / align);
const ALIGN: u64 = virtual_file::get_io_buffer_alignment() as u64;
assert_eq!(read.start % ALIGN, 0);
assert_eq!(read.start / ALIGN, offset_range.first().unwrap().2 / ALIGN);
let expected_offsets_in_read: Vec<_> = offset_range.iter().map(|o| o.2).collect();
@@ -676,32 +657,27 @@ mod tests {
fn planner_chunked_coalesce_all_test() {
use crate::virtual_file;
let chunk_size = virtual_file::get_io_buffer_alignment() as u64;
const CHUNK_SIZE: u64 = virtual_file::get_io_buffer_alignment() as u64;
// The test explicitly does not check chunk size < 512
if chunk_size < 512 {
return;
}
let max_read_size = chunk_size as usize * 8;
let max_read_size = CHUNK_SIZE as usize * 8;
let key = Key::MIN;
let lsn = Lsn(0);
let blob_descriptions = [
(key, lsn, chunk_size / 8, BlobFlag::None), // Read 1 BEGIN
(key, lsn, chunk_size / 4, BlobFlag::Ignore), // Gap
(key, lsn, chunk_size / 2, BlobFlag::None),
(key, lsn, chunk_size - 2, BlobFlag::Ignore), // Gap
(key, lsn, chunk_size, BlobFlag::None),
(key, lsn, chunk_size * 2 - 1, BlobFlag::None),
(key, lsn, chunk_size * 2 + 1, BlobFlag::Ignore), // Gap
(key, lsn, chunk_size * 3 + 1, BlobFlag::None),
(key, lsn, chunk_size * 5 + 1, BlobFlag::None),
(key, lsn, chunk_size * 6 + 1, BlobFlag::Ignore), // skipped chunk size, but not a chunk: should coalesce.
(key, lsn, chunk_size * 7 + 1, BlobFlag::None),
(key, lsn, chunk_size * 8, BlobFlag::None), // Read 2 BEGIN (b/c max_read_size)
(key, lsn, chunk_size * 9, BlobFlag::Ignore), // ==== skipped a chunk
(key, lsn, chunk_size * 10, BlobFlag::None), // Read 3 BEGIN (cannot coalesce)
(key, lsn, CHUNK_SIZE / 8, BlobFlag::None), // Read 1 BEGIN
(key, lsn, CHUNK_SIZE / 4, BlobFlag::Ignore), // Gap
(key, lsn, CHUNK_SIZE / 2, BlobFlag::None),
(key, lsn, CHUNK_SIZE - 2, BlobFlag::Ignore), // Gap
(key, lsn, CHUNK_SIZE, BlobFlag::None),
(key, lsn, CHUNK_SIZE * 2 - 1, BlobFlag::None),
(key, lsn, CHUNK_SIZE * 2 + 1, BlobFlag::Ignore), // Gap
(key, lsn, CHUNK_SIZE * 3 + 1, BlobFlag::None),
(key, lsn, CHUNK_SIZE * 5 + 1, BlobFlag::None),
(key, lsn, CHUNK_SIZE * 6 + 1, BlobFlag::Ignore), // skipped chunk size, but not a chunk: should coalesce.
(key, lsn, CHUNK_SIZE * 7 + 1, BlobFlag::None),
(key, lsn, CHUNK_SIZE * 8, BlobFlag::None), // Read 2 BEGIN (b/c max_read_size)
(key, lsn, CHUNK_SIZE * 9, BlobFlag::Ignore), // ==== skipped a chunk
(key, lsn, CHUNK_SIZE * 10, BlobFlag::None), // Read 3 BEGIN (cannot coalesce)
];
let ranges = [
@@ -780,19 +756,19 @@ mod tests {
#[test]
fn planner_replacement_test() {
let chunk_size = virtual_file::get_io_buffer_alignment() as u64;
let max_read_size = 128 * chunk_size as usize;
const CHUNK_SIZE: u64 = virtual_file::get_io_buffer_alignment() as u64;
let max_read_size = 128 * CHUNK_SIZE as usize;
let first_key = Key::MIN;
let second_key = first_key.next();
let lsn = Lsn(0);
let blob_descriptions = vec![
(first_key, lsn, 0, BlobFlag::None), // First in read 1
(first_key, lsn, chunk_size, BlobFlag::None), // Last in read 1
(second_key, lsn, 2 * chunk_size, BlobFlag::ReplaceAll),
(second_key, lsn, 3 * chunk_size, BlobFlag::None),
(second_key, lsn, 4 * chunk_size, BlobFlag::ReplaceAll), // First in read 2
(second_key, lsn, 5 * chunk_size, BlobFlag::None), // Last in read 2
(first_key, lsn, CHUNK_SIZE, BlobFlag::None), // Last in read 1
(second_key, lsn, 2 * CHUNK_SIZE, BlobFlag::ReplaceAll),
(second_key, lsn, 3 * CHUNK_SIZE, BlobFlag::None),
(second_key, lsn, 4 * CHUNK_SIZE, BlobFlag::ReplaceAll), // First in read 2
(second_key, lsn, 5 * CHUNK_SIZE, BlobFlag::None), // Last in read 2
];
let ranges = [&blob_descriptions[0..2], &blob_descriptions[4..]];
@@ -802,7 +778,7 @@ mod tests {
planner.handle(key, lsn, offset, flag);
}
planner.handle_range_end(6 * chunk_size);
planner.handle_range_end(6 * CHUNK_SIZE);
let reads = planner.finish();
assert_eq!(reads.len(), 2);
@@ -947,7 +923,6 @@ mod tests {
let reserved_bytes = blobs.iter().map(|bl| bl.len()).max().unwrap() * 2 + 16;
let mut buf = BytesMut::with_capacity(reserved_bytes);
let align = virtual_file::get_io_buffer_alignment();
let vectored_blob_reader = VectoredBlobReader::new(&file);
let meta = BlobMeta {
key: Key::MIN,
@@ -959,8 +934,7 @@ mod tests {
if idx + 1 == offsets.len() {
continue;
}
let read_builder =
ChunkedVectoredReadBuilder::new(*offset, *end, meta, 16 * 4096, align);
let read_builder = ChunkedVectoredReadBuilder::new(*offset, *end, meta, 16 * 4096);
let read = read_builder.build();
let result = vectored_blob_reader.read_blobs(&read, buf, &ctx).await?;
assert_eq!(result.blobs.len(), 1);

View File

@@ -1330,15 +1330,10 @@ impl OpenFiles {
/// server startup.
///
#[cfg(not(test))]
pub fn init(num_slots: usize, engine: IoEngineKind, io_buffer_alignment: usize) {
pub fn init(num_slots: usize, engine: IoEngineKind) {
if OPEN_FILES.set(OpenFiles::new(num_slots)).is_err() {
panic!("virtual_file::init called twice");
}
if set_io_buffer_alignment(io_buffer_alignment).is_err() {
panic!(
"IO buffer alignment needs to be a power of two and greater than 512, got {io_buffer_alignment}"
);
}
io_engine::init(engine);
crate::metrics::virtual_file_descriptor_cache::SIZE_MAX.set(num_slots as u64);
}
@@ -1362,45 +1357,9 @@ fn get_open_files() -> &'static OpenFiles {
}
}
static IO_BUFFER_ALIGNMENT: AtomicUsize = AtomicUsize::new(DEFAULT_IO_BUFFER_ALIGNMENT);
/// Returns true if the alignment is a power of two and is greater or equal to 512.
fn is_valid_io_buffer_alignment(align: usize) -> bool {
align.is_power_of_two() && align >= 512
}
/// Sets IO buffer alignment requirement. Returns error if the alignment requirement is
/// not a power of two or less than 512 bytes.
#[allow(unused)]
pub(crate) fn set_io_buffer_alignment(align: usize) -> Result<(), usize> {
if is_valid_io_buffer_alignment(align) {
IO_BUFFER_ALIGNMENT.store(align, std::sync::atomic::Ordering::Relaxed);
Ok(())
} else {
Err(align)
}
}
/// Gets the io buffer alignment.
///
/// This function should be used for getting the actual alignment value to use.
pub(crate) fn get_io_buffer_alignment() -> usize {
let align = IO_BUFFER_ALIGNMENT.load(std::sync::atomic::Ordering::Relaxed);
if cfg!(test) {
let env_var_name = "NEON_PAGESERVER_UNIT_TEST_IO_BUFFER_ALIGNMENT";
if let Some(test_align) = utils::env::var(env_var_name) {
if is_valid_io_buffer_alignment(test_align) {
test_align
} else {
panic!("IO buffer alignment needs to be a power of two and greater than 512, got {test_align}");
}
} else {
align
}
} else {
align
}
pub(crate) const fn get_io_buffer_alignment() -> usize {
DEFAULT_IO_BUFFER_ALIGNMENT
}
static IO_MODE: AtomicU8 = AtomicU8::new(IoMode::preferred() as u8);

View File

@@ -398,7 +398,6 @@ class NeonEnvBuilder:
pageserver_default_tenant_config_compaction_algorithm: Optional[Dict[str, Any]] = None,
safekeeper_extra_opts: Optional[list[str]] = None,
storage_controller_port_override: Optional[int] = None,
pageserver_io_buffer_alignment: Optional[int] = None,
pageserver_virtual_file_io_mode: Optional[str] = None,
):
self.repo_dir = repo_dir
@@ -453,7 +452,6 @@ class NeonEnvBuilder:
self.storage_controller_port_override = storage_controller_port_override
self.pageserver_io_buffer_alignment = pageserver_io_buffer_alignment
self.pageserver_virtual_file_io_mode = pageserver_virtual_file_io_mode
assert test_name.startswith(
@@ -1043,7 +1041,6 @@ class NeonEnv:
self.pageserver_virtual_file_io_engine = config.pageserver_virtual_file_io_engine
self.pageserver_aux_file_policy = config.pageserver_aux_file_policy
self.pageserver_io_buffer_alignment = config.pageserver_io_buffer_alignment
self.pageserver_virtual_file_io_mode = config.pageserver_virtual_file_io_mode
# Create the neon_local's `NeonLocalInitConf`
@@ -1108,8 +1105,6 @@ class NeonEnv:
for key, value in override.items():
ps_cfg[key] = value
if self.pageserver_io_buffer_alignment is not None:
ps_cfg["io_buffer_alignment"] = self.pageserver_io_buffer_alignment
if self.pageserver_virtual_file_io_mode is not None:
ps_cfg["virtual_file_io_mode"] = self.pageserver_virtual_file_io_mode
@@ -1417,7 +1412,6 @@ def neon_simple_env(
pageserver_virtual_file_io_engine: str,
pageserver_aux_file_policy: Optional[AuxFileStore],
pageserver_default_tenant_config_compaction_algorithm: Optional[Dict[str, Any]],
pageserver_io_buffer_alignment: Optional[int],
pageserver_virtual_file_io_mode: Optional[str],
) -> Iterator[NeonEnv]:
"""
@@ -1444,7 +1438,6 @@ def neon_simple_env(
pageserver_virtual_file_io_engine=pageserver_virtual_file_io_engine,
pageserver_aux_file_policy=pageserver_aux_file_policy,
pageserver_default_tenant_config_compaction_algorithm=pageserver_default_tenant_config_compaction_algorithm,
pageserver_io_buffer_alignment=pageserver_io_buffer_alignment,
pageserver_virtual_file_io_mode=pageserver_virtual_file_io_mode,
) as builder:
env = builder.init_start()
@@ -1469,7 +1462,6 @@ def neon_env_builder(
pageserver_default_tenant_config_compaction_algorithm: Optional[Dict[str, Any]],
pageserver_aux_file_policy: Optional[AuxFileStore],
record_property: Callable[[str, object], None],
pageserver_io_buffer_alignment: Optional[int],
pageserver_virtual_file_io_mode: Optional[str],
) -> Iterator[NeonEnvBuilder]:
"""
@@ -1505,7 +1497,6 @@ def neon_env_builder(
test_overlay_dir=test_overlay_dir,
pageserver_aux_file_policy=pageserver_aux_file_policy,
pageserver_default_tenant_config_compaction_algorithm=pageserver_default_tenant_config_compaction_algorithm,
pageserver_io_buffer_alignment=pageserver_io_buffer_alignment,
pageserver_virtual_file_io_mode=pageserver_virtual_file_io_mode,
) as builder:
yield builder

View File

@@ -34,11 +34,6 @@ def pageserver_virtual_file_io_engine() -> Optional[str]:
return os.getenv("PAGESERVER_VIRTUAL_FILE_IO_ENGINE")
@pytest.fixture(scope="function", autouse=True)
def pageserver_io_buffer_alignment() -> Optional[int]:
return None
@pytest.fixture(scope="function", autouse=True)
def pageserver_virtual_file_io_mode() -> Optional[str]:
return os.getenv("PAGESERVER_VIRTUAL_FILE_IO_MODE")