Eliminate dependency from pageserver_api to postgres_ffi (#12273)

Introduce a separate `postgres_ffi_types` crate which contains a few
types and functions that were used in the API. `postgres_ffi_types` is a
much small crate than `postgres_ffi`, and it doesn't depend on bindgen
or the Postgres C headers.

Move NeonWalRecord and Value types to wal_decoder crate. They are only
used in the pageserver-safekeeper "ingest" API. The rest of the ingest
API types are defined in wal_decoder, so move these there as well.
This commit is contained in:
Heikki Linnakangas
2025-06-19 13:31:27 +03:00
committed by GitHub
parent 2ca6665f4a
commit 1950ccfe33
44 changed files with 183 additions and 128 deletions

13
Cargo.lock generated
View File

@@ -4334,6 +4334,7 @@ dependencies = [
"postgres_backend",
"postgres_connection",
"postgres_ffi",
"postgres_ffi_types",
"postgres_initdb",
"posthog_client_lite",
"pprof",
@@ -4403,7 +4404,7 @@ dependencies = [
"nix 0.30.1",
"once_cell",
"postgres_backend",
"postgres_ffi",
"postgres_ffi_types",
"rand 0.8.5",
"remote_storage",
"reqwest",
@@ -4892,6 +4893,7 @@ dependencies = [
"memoffset 0.9.0",
"once_cell",
"postgres",
"postgres_ffi_types",
"pprof",
"regex",
"serde",
@@ -4900,6 +4902,14 @@ dependencies = [
"utils",
]
[[package]]
name = "postgres_ffi_types"
version = "0.1.0"
dependencies = [
"thiserror 1.0.69",
"workspace_hack",
]
[[package]]
name = "postgres_initdb"
version = "0.1.0"
@@ -8161,6 +8171,7 @@ dependencies = [
"futures",
"pageserver_api",
"postgres_ffi",
"postgres_ffi_types",
"pprof",
"prost 0.13.5",
"remote_storage",

View File

@@ -22,6 +22,7 @@ members = [
"libs/http-utils",
"libs/pageserver_api",
"libs/postgres_ffi",
"libs/postgres_ffi_types",
"libs/safekeeper_api",
"libs/desim",
"libs/neon-shmem",
@@ -259,6 +260,7 @@ pageserver_page_api = { path = "./pageserver/page_api" }
postgres_backend = { version = "0.1", path = "./libs/postgres_backend/" }
postgres_connection = { version = "0.1", path = "./libs/postgres_connection/" }
postgres_ffi = { version = "0.1", path = "./libs/postgres_ffi/" }
postgres_ffi_types = { version = "0.1", path = "./libs/postgres_ffi_types/" }
postgres_initdb = { path = "./libs/postgres_initdb" }
posthog_client_lite = { version = "0.1", path = "./libs/posthog_client_lite" }
pq_proto = { version = "0.1", path = "./libs/pq_proto/" }

View File

@@ -17,7 +17,7 @@ anyhow.workspace = true
bytes.workspace = true
byteorder.workspace = true
utils.workspace = true
postgres_ffi.workspace = true
postgres_ffi_types.workspace = true
enum-map.workspace = true
strum.workspace = true
strum_macros.workspace = true

View File

@@ -4,8 +4,8 @@ use std::ops::Range;
use anyhow::{Result, bail};
use byteorder::{BE, ByteOrder};
use bytes::Bytes;
use postgres_ffi::relfile_utils::{FSM_FORKNUM, VISIBILITYMAP_FORKNUM};
use postgres_ffi::{Oid, RepOriginId};
use postgres_ffi_types::forknum::{FSM_FORKNUM, VISIBILITYMAP_FORKNUM};
use postgres_ffi_types::{Oid, RepOriginId};
use serde::{Deserialize, Serialize};
use utils::const_assert;
@@ -194,7 +194,7 @@ impl Key {
/// will be rejected on the write path.
#[allow(dead_code)]
pub fn is_valid_key_on_write_path_strong(&self) -> bool {
use postgres_ffi::pg_constants::{DEFAULTTABLESPACE_OID, GLOBALTABLESPACE_OID};
use postgres_ffi_types::constants::{DEFAULTTABLESPACE_OID, GLOBALTABLESPACE_OID};
if !self.is_i128_representable() {
return false;
}

View File

@@ -1,7 +1,6 @@
use std::ops::Range;
use itertools::Itertools;
use postgres_ffi::BLCKSZ;
use crate::key::Key;
use crate::shard::{ShardCount, ShardIdentity};
@@ -269,9 +268,13 @@ impl KeySpace {
/// Partition a key space into roughly chunks of roughly 'target_size' bytes
/// in each partition.
///
pub fn partition(&self, shard_identity: &ShardIdentity, target_size: u64) -> KeyPartitioning {
// Assume that each value is 8k in size.
let target_nblocks = (target_size / BLCKSZ as u64) as u32;
pub fn partition(
&self,
shard_identity: &ShardIdentity,
target_size: u64,
block_size: u64,
) -> KeyPartitioning {
let target_nblocks = (target_size / block_size) as u32;
let mut parts = Vec::new();
let mut current_part = Vec::new();

View File

@@ -6,11 +6,9 @@ pub mod key;
pub mod keyspace;
pub mod models;
pub mod pagestream_api;
pub mod record;
pub mod reltag;
pub mod shard;
/// Public API types
pub mod upcall_api;
pub mod value;
pub mod config;

View File

@@ -8,9 +8,15 @@ use crate::reltag::RelTag;
use byteorder::{BigEndian, ReadBytesExt};
use bytes::{Buf, BufMut, Bytes, BytesMut};
use postgres_ffi::BLCKSZ;
use utils::lsn::Lsn;
/// Block size.
///
/// XXX: We assume 8k block size in the SLRU fetch API. It's not great to hardcode
/// that in the protocol, because Postgres supports different block sizes as a compile
/// time option.
const BLCKSZ: usize = 8192;
// Wrapped in libpq CopyData
#[derive(PartialEq, Eq, Debug)]
pub enum PagestreamFeMessage {
@@ -443,7 +449,7 @@ impl PagestreamBeMessage {
Self::GetSlruSegment(resp) => {
bytes.put_u8(Tag::GetSlruSegment as u8);
bytes.put_u32((resp.segment.len() / BLCKSZ as usize) as u32);
bytes.put_u32((resp.segment.len() / BLCKSZ) as u32);
bytes.put(&resp.segment[..]);
}
@@ -520,7 +526,7 @@ impl PagestreamBeMessage {
bytes.put_u64(resp.req.hdr.not_modified_since.0);
bytes.put_u8(resp.req.kind);
bytes.put_u32(resp.req.segno);
bytes.put_u32((resp.segment.len() / BLCKSZ as usize) as u32);
bytes.put_u32((resp.segment.len() / BLCKSZ) as u32);
bytes.put(&resp.segment[..]);
}
@@ -662,7 +668,7 @@ impl PagestreamBeMessage {
let kind = buf.read_u8()?;
let segno = buf.read_u32::<BigEndian>()?;
let n_blocks = buf.read_u32::<BigEndian>()?;
let mut segment = vec![0; n_blocks as usize * BLCKSZ as usize];
let mut segment = vec![0; n_blocks as usize * BLCKSZ];
buf.read_exact(&mut segment)?;
Self::GetSlruSegment(PagestreamGetSlruSegmentResponse {
req: PagestreamGetSlruSegmentRequest {

View File

@@ -1,9 +1,9 @@
use std::cmp::Ordering;
use std::fmt;
use postgres_ffi::Oid;
use postgres_ffi::pg_constants::GLOBALTABLESPACE_OID;
use postgres_ffi::relfile_utils::{MAIN_FORKNUM, forkname_to_number, forknumber_to_name};
use postgres_ffi_types::Oid;
use postgres_ffi_types::constants::GLOBALTABLESPACE_OID;
use postgres_ffi_types::forknum::{MAIN_FORKNUM, forkname_to_number, forknumber_to_name};
use serde::{Deserialize, Serialize};
///

View File

@@ -35,7 +35,7 @@ use std::hash::{Hash, Hasher};
#[doc(inline)]
pub use ::utils::shard::*;
use postgres_ffi::relfile_utils::INIT_FORKNUM;
use postgres_ffi_types::forknum::INIT_FORKNUM;
use serde::{Deserialize, Serialize};
use crate::key::Key;

View File

@@ -16,6 +16,7 @@ memoffset.workspace = true
pprof.workspace = true
thiserror.workspace = true
serde.workspace = true
postgres_ffi_types.workspace = true
utils.workspace = true
tracing.workspace = true

View File

@@ -11,11 +11,7 @@
use crate::{BLCKSZ, PageHeaderData};
//
// From pg_tablespace_d.h
//
pub const DEFAULTTABLESPACE_OID: u32 = 1663;
pub const GLOBALTABLESPACE_OID: u32 = 1664;
// Note: There are a few more widely-used constants in the postgres_ffi_types::constants crate.
// From storage_xlog.h
pub const XLOG_SMGR_CREATE: u8 = 0x10;

View File

@@ -4,50 +4,7 @@
use once_cell::sync::OnceCell;
use regex::Regex;
//
// Fork numbers, from relpath.h
//
pub const MAIN_FORKNUM: u8 = 0;
pub const FSM_FORKNUM: u8 = 1;
pub const VISIBILITYMAP_FORKNUM: u8 = 2;
pub const INIT_FORKNUM: u8 = 3;
#[derive(Debug, Clone, thiserror::Error, PartialEq, Eq)]
pub enum FilePathError {
#[error("invalid relation fork name")]
InvalidForkName,
#[error("invalid relation data file name")]
InvalidFileName,
}
impl From<core::num::ParseIntError> for FilePathError {
fn from(_e: core::num::ParseIntError) -> Self {
FilePathError::InvalidFileName
}
}
/// Convert Postgres relation file's fork suffix to fork number.
pub fn forkname_to_number(forkname: Option<&str>) -> Result<u8, FilePathError> {
match forkname {
// "main" is not in filenames, it's implicit if the fork name is not present
None => Ok(MAIN_FORKNUM),
Some("fsm") => Ok(FSM_FORKNUM),
Some("vm") => Ok(VISIBILITYMAP_FORKNUM),
Some("init") => Ok(INIT_FORKNUM),
Some(_) => Err(FilePathError::InvalidForkName),
}
}
/// Convert Postgres fork number to the right suffix of the relation data file.
pub fn forknumber_to_name(forknum: u8) -> Option<&'static str> {
match forknum {
MAIN_FORKNUM => None,
FSM_FORKNUM => Some("fsm"),
VISIBILITYMAP_FORKNUM => Some("vm"),
INIT_FORKNUM => Some("init"),
_ => Some("UNKNOWN FORKNUM"),
}
}
use postgres_ffi_types::forknum::*;
/// Parse a filename of a relation file. Returns (relfilenode, forknum, segno) tuple.
///
@@ -75,7 +32,9 @@ pub fn parse_relfilename(fname: &str) -> Result<(u32, u8, u32), FilePathError> {
.ok_or(FilePathError::InvalidFileName)?;
let relnode_str = caps.name("relnode").unwrap().as_str();
let relnode = relnode_str.parse::<u32>()?;
let relnode = relnode_str
.parse::<u32>()
.map_err(|_e| FilePathError::InvalidFileName)?;
let forkname = caps.name("forkname").map(|f| f.as_str());
let forknum = forkname_to_number(forkname)?;
@@ -84,7 +43,11 @@ pub fn parse_relfilename(fname: &str) -> Result<(u32, u8, u32), FilePathError> {
let segno = if segno_match.is_none() {
0
} else {
segno_match.unwrap().as_str().parse::<u32>()?
segno_match
.unwrap()
.as_str()
.parse::<u32>()
.map_err(|_e| FilePathError::InvalidFileName)?
};
Ok((relnode, forknum, segno))

View File

@@ -0,0 +1,11 @@
[package]
name = "postgres_ffi_types"
version = "0.1.0"
edition.workspace = true
license.workspace = true
[dependencies]
thiserror.workspace = true
workspace_hack = { version = "0.1", path = "../../workspace_hack" }
[dev-dependencies]

View File

@@ -0,0 +1,8 @@
//! Misc constants, copied from PostgreSQL headers.
//!
//! Any constants included here must be the same in all PostgreSQL versions and unlikely to change
//! in the future either!
// From pg_tablespace_d.h
pub const DEFAULTTABLESPACE_OID: u32 = 1663;
pub const GLOBALTABLESPACE_OID: u32 = 1664;

View File

@@ -0,0 +1,36 @@
// Fork numbers, from relpath.h
pub const MAIN_FORKNUM: u8 = 0;
pub const FSM_FORKNUM: u8 = 1;
pub const VISIBILITYMAP_FORKNUM: u8 = 2;
pub const INIT_FORKNUM: u8 = 3;
#[derive(Debug, Clone, thiserror::Error, PartialEq, Eq)]
pub enum FilePathError {
#[error("invalid relation fork name")]
InvalidForkName,
#[error("invalid relation data file name")]
InvalidFileName,
}
/// Convert Postgres relation file's fork suffix to fork number.
pub fn forkname_to_number(forkname: Option<&str>) -> Result<u8, FilePathError> {
match forkname {
// "main" is not in filenames, it's implicit if the fork name is not present
None => Ok(MAIN_FORKNUM),
Some("fsm") => Ok(FSM_FORKNUM),
Some("vm") => Ok(VISIBILITYMAP_FORKNUM),
Some("init") => Ok(INIT_FORKNUM),
Some(_) => Err(FilePathError::InvalidForkName),
}
}
/// Convert Postgres fork number to the right suffix of the relation data file.
pub fn forknumber_to_name(forknum: u8) -> Option<&'static str> {
match forknum {
MAIN_FORKNUM => None,
FSM_FORKNUM => Some("fsm"),
VISIBILITYMAP_FORKNUM => Some("vm"),
INIT_FORKNUM => Some("init"),
_ => Some("UNKNOWN FORKNUM"),
}
}

View File

@@ -0,0 +1,13 @@
//! This package contains some PostgreSQL constants and datatypes that are the same in all versions
//! of PostgreSQL and unlikely to change in the future either. These could be derived from the
//! PostgreSQL headers with 'bindgen', but in order to avoid proliferating the dependency to bindgen
//! and the PostgreSQL C headers to all services, we prefer to have this small stand-alone crate for
//! them instead.
//!
//! Be mindful in what you add here, as these types are deeply ingrained in the APIs.
pub mod constants;
pub mod forknum;
pub type Oid = u32;
pub type RepOriginId = u16;

View File

@@ -14,6 +14,7 @@ bytes.workspace = true
pageserver_api.workspace = true
prost.workspace = true
postgres_ffi.workspace = true
postgres_ffi_types.workspace = true
serde.workspace = true
thiserror.workspace = true
tokio = { workspace = true, features = ["io-util"] }

View File

@@ -8,8 +8,8 @@ use pageserver_api::key::rel_block_to_key;
use pageserver_api::reltag::{RelTag, SlruKind};
use pageserver_api::shard::ShardIdentity;
use postgres_ffi::pg_constants;
use postgres_ffi::relfile_utils::VISIBILITYMAP_FORKNUM;
use postgres_ffi::walrecord::*;
use postgres_ffi_types::forknum::VISIBILITYMAP_FORKNUM;
use utils::lsn::Lsn;
use crate::models::*;

View File

@@ -25,6 +25,9 @@
//! |
//! |--> write to KV store within the pageserver
pub mod record;
pub mod value;
use bytes::Bytes;
use pageserver_api::reltag::{RelTag, SlruKind};
use postgres_ffi::walrecord::{

View File

@@ -10,7 +10,7 @@
use bytes::Bytes;
use serde::{Deserialize, Serialize};
use crate::record::NeonWalRecord;
use crate::models::record::NeonWalRecord;
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub enum Value {

View File

@@ -1,4 +1,4 @@
//! This module implements batch type for serialized [`pageserver_api::value::Value`]
//! This module implements batch type for serialized [`crate::models::value::Value`]
//! instances. Each batch contains a raw buffer (serialized values)
//! and a list of metadata for each (key, LSN) tuple present in the batch.
//!
@@ -10,10 +10,8 @@ use std::collections::{BTreeSet, HashMap};
use bytes::{Bytes, BytesMut};
use pageserver_api::key::{CompactKey, Key, rel_block_to_key};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::RelTag;
use pageserver_api::shard::ShardIdentity;
use pageserver_api::value::Value;
use postgres_ffi::walrecord::{DecodedBkpBlock, DecodedWALRecord};
use postgres_ffi::{BLCKSZ, page_is_new, page_set_lsn, pg_constants};
use serde::{Deserialize, Serialize};
@@ -21,6 +19,8 @@ use utils::bin_ser::BeSer;
use utils::lsn::Lsn;
use crate::models::InterpretedWalRecord;
use crate::models::record::NeonWalRecord;
use crate::models::value::Value;
static ZERO_PAGE: Bytes = Bytes::from_static(&[0u8; BLCKSZ as usize]);

View File

@@ -56,6 +56,7 @@ pin-project-lite.workspace = true
postgres_backend.workspace = true
postgres_connection.workspace = true
postgres_ffi.workspace = true
postgres_ffi_types.workspace = true
postgres_initdb.workspace = true
postgres-protocol.workspace = true
postgres-types.workspace = true

View File

@@ -13,11 +13,11 @@ use pageserver::{page_cache, virtual_file};
use pageserver_api::key::Key;
use pageserver_api::models::virtual_file::IoMode;
use pageserver_api::shard::TenantShardId;
use pageserver_api::value::Value;
use strum::IntoEnumIterator;
use tokio_util::sync::CancellationToken;
use utils::bin_ser::BeSer;
use utils::id::{TenantId, TimelineId};
use wal_decoder::models::value::Value;
use wal_decoder::serialized_batch::SerializedValueBatch;
// A very cheap hash for generating non-sequential keys.

View File

@@ -67,12 +67,12 @@ use once_cell::sync::Lazy;
use pageserver::config::PageServerConf;
use pageserver::walredo::{PostgresRedoManager, RedoAttemptType};
use pageserver_api::key::Key;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::TenantShardId;
use tokio::sync::Barrier;
use tokio::task::JoinSet;
use utils::id::TenantId;
use utils::lsn::Lsn;
use wal_decoder::models::record::NeonWalRecord;
fn bench(c: &mut Criterion) {
macro_rules! bench_group {

View File

@@ -18,13 +18,12 @@ use bytes::{BufMut, Bytes, BytesMut};
use fail::fail_point;
use pageserver_api::key::{Key, rel_block_to_key};
use pageserver_api::reltag::{RelTag, SlruKind};
use postgres_ffi::pg_constants::{
DEFAULTTABLESPACE_OID, GLOBALTABLESPACE_OID, PG_HBA, PGDATA_SPECIAL_FILES,
};
use postgres_ffi::relfile_utils::{INIT_FORKNUM, MAIN_FORKNUM};
use postgres_ffi::pg_constants::{PG_HBA, PGDATA_SPECIAL_FILES};
use postgres_ffi::{
BLCKSZ, PG_TLI, RELSEG_SIZE, WAL_SEGMENT_SIZE, XLogFileName, dispatch_pgversion, pg_constants,
};
use postgres_ffi_types::constants::{DEFAULTTABLESPACE_OID, GLOBALTABLESPACE_OID};
use postgres_ffi_types::forknum::{INIT_FORKNUM, MAIN_FORKNUM};
use tokio::io;
use tokio::io::AsyncWrite;
use tokio_tar::{Builder, EntryType, Header};
@@ -372,6 +371,7 @@ where
.partition(
self.timeline.get_shard_identity(),
self.timeline.conf.max_get_vectored_keys.get() as u64 * BLCKSZ as u64,
BLCKSZ as u64,
);
let mut slru_builder = SlruSegmentsBuilder::new(&mut self.ar);

View File

@@ -520,7 +520,7 @@ async fn import_file(
}
if file_path.starts_with("global") {
let spcnode = postgres_ffi::pg_constants::GLOBALTABLESPACE_OID;
let spcnode = postgres_ffi_types::constants::GLOBALTABLESPACE_OID;
let dbnode = 0;
match file_name.as_ref() {
@@ -553,7 +553,7 @@ async fn import_file(
}
}
} else if file_path.starts_with("base") {
let spcnode = pg_constants::DEFAULTTABLESPACE_OID;
let spcnode = postgres_ffi_types::constants::DEFAULTTABLESPACE_OID;
let dbnode: u32 = file_path
.iter()
.nth(1)

View File

@@ -41,7 +41,7 @@ use postgres_backend::{
AuthType, PostgresBackend, PostgresBackendReader, QueryError, is_expected_io_error,
};
use postgres_ffi::BLCKSZ;
use postgres_ffi::pg_constants::DEFAULTTABLESPACE_OID;
use postgres_ffi_types::constants::DEFAULTTABLESPACE_OID;
use pq_proto::framed::ConnectionError;
use pq_proto::{BeMessage, FeMessage, FeStartupPacket, RowDescriptor};
use smallvec::{SmallVec, smallvec};

View File

@@ -23,12 +23,11 @@ use pageserver_api::key::{
};
use pageserver_api::keyspace::{KeySpaceRandomAccum, SparseKeySpace};
use pageserver_api::models::RelSizeMigration;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::{BlockNumber, RelTag, SlruKind};
use pageserver_api::shard::ShardIdentity;
use pageserver_api::value::Value;
use postgres_ffi::relfile_utils::{FSM_FORKNUM, VISIBILITYMAP_FORKNUM};
use postgres_ffi::{BLCKSZ, Oid, RepOriginId, TimestampTz, TransactionId};
use postgres_ffi::{BLCKSZ, TimestampTz, TransactionId};
use postgres_ffi_types::forknum::{FSM_FORKNUM, VISIBILITYMAP_FORKNUM};
use postgres_ffi_types::{Oid, RepOriginId};
use serde::{Deserialize, Serialize};
use strum::IntoEnumIterator;
use tokio_util::sync::CancellationToken;
@@ -36,6 +35,8 @@ use tracing::{debug, info, info_span, trace, warn};
use utils::bin_ser::{BeSer, DeserializeError};
use utils::lsn::Lsn;
use utils::pausable_failpoint;
use wal_decoder::models::record::NeonWalRecord;
use wal_decoder::models::value::Value;
use wal_decoder::serialized_batch::{SerializedValueBatch, ValueMeta};
use super::tenant::{PageReconstructError, Timeline};
@@ -720,6 +721,7 @@ impl Timeline {
let batches = keyspace.partition(
self.get_shard_identity(),
self.conf.max_get_vectored_keys.get() as u64 * BLCKSZ as u64,
BLCKSZ as u64,
);
let io_concurrency = IoConcurrency::spawn_from_conf(
@@ -960,6 +962,7 @@ impl Timeline {
let batches = keyspace.partition(
self.get_shard_identity(),
self.conf.max_get_vectored_keys.get() as u64 * BLCKSZ as u64,
BLCKSZ as u64,
);
let io_concurrency = IoConcurrency::spawn_from_conf(

View File

@@ -496,7 +496,7 @@ impl WalRedoManager {
key: pageserver_api::key::Key,
lsn: Lsn,
base_img: Option<(Lsn, bytes::Bytes)>,
records: Vec<(Lsn, pageserver_api::record::NeonWalRecord)>,
records: Vec<(Lsn, wal_decoder::models::record::NeonWalRecord)>,
pg_version: u32,
redo_attempt_type: RedoAttemptType,
) -> Result<bytes::Bytes, walredo::Error> {
@@ -5852,10 +5852,10 @@ pub(crate) mod harness {
use once_cell::sync::OnceCell;
use pageserver_api::key::Key;
use pageserver_api::models::ShardParameters;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::ShardIndex;
use utils::id::TenantId;
use utils::logging;
use wal_decoder::models::record::NeonWalRecord;
use super::*;
use crate::deletion_queue::mock::MockDeletionQueue;
@@ -6110,9 +6110,6 @@ mod tests {
#[cfg(feature = "testing")]
use pageserver_api::keyspace::KeySpaceRandomAccum;
use pageserver_api::models::{CompactionAlgorithm, CompactionAlgorithmSettings};
#[cfg(feature = "testing")]
use pageserver_api::record::NeonWalRecord;
use pageserver_api::value::Value;
use pageserver_compaction::helpers::overlaps_with;
#[cfg(feature = "testing")]
use rand::SeedableRng;
@@ -6133,6 +6130,9 @@ mod tests {
use timeline::{CompactOptions, DeltaLayerTestDesc, VersionedKeySpaceQuery};
use utils::id::TenantId;
use utils::shard::{ShardCount, ShardNumber};
#[cfg(feature = "testing")]
use wal_decoder::models::record::NeonWalRecord;
use wal_decoder::models::value::Value;
use super::*;
use crate::DEFAULT_PG_VERSION;

View File

@@ -34,11 +34,11 @@ pub use layer_name::{DeltaLayerName, ImageLayerName, LayerName};
use pageserver_api::config::GetVectoredConcurrentIo;
use pageserver_api::key::Key;
use pageserver_api::keyspace::{KeySpace, KeySpaceRandomAccum};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::value::Value;
use tracing::{Instrument, info_span, trace};
use utils::lsn::Lsn;
use utils::sync::gate::GateGuard;
use wal_decoder::models::record::NeonWalRecord;
use wal_decoder::models::value::Value;
use self::inmemory_layer::InMemoryLayerFileId;
use super::PageReconstructError;

View File

@@ -4,11 +4,11 @@ use std::sync::Arc;
use bytes::Bytes;
use pageserver_api::key::{KEY_SIZE, Key};
use pageserver_api::value::Value;
use tokio_util::sync::CancellationToken;
use utils::id::TimelineId;
use utils::lsn::Lsn;
use utils::shard::TenantShardId;
use wal_decoder::models::value::Value;
use super::errors::PutError;
use super::layer::S3_UPLOAD_LIMIT;

View File

@@ -44,7 +44,6 @@ use pageserver_api::key::{DBDIR_KEY, KEY_SIZE, Key};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::models::ImageCompressionAlgorithm;
use pageserver_api::shard::TenantShardId;
use pageserver_api::value::Value;
use serde::{Deserialize, Serialize};
use tokio::sync::OnceCell;
use tokio_epoll_uring::IoBuf;
@@ -54,6 +53,7 @@ use utils::bin_ser::BeSer;
use utils::bin_ser::SerializeError;
use utils::id::{TenantId, TimelineId};
use utils::lsn::Lsn;
use wal_decoder::models::value::Value;
use super::errors::PutError;
use super::{
@@ -1306,7 +1306,7 @@ impl DeltaLayerInner {
// is it an image or will_init walrecord?
// FIXME: this could be handled by threading the BlobRef to the
// VectoredReadBuilder
let will_init = pageserver_api::value::ValueBytes::will_init(&data)
let will_init = wal_decoder::models::value::ValueBytes::will_init(&data)
.inspect_err(|_e| {
#[cfg(feature = "testing")]
tracing::error!(data=?utils::Hex(&data), err=?_e, %key, %lsn, "failed to parse will_init out of serialized value");
@@ -1369,7 +1369,7 @@ impl DeltaLayerInner {
format!(" img {} bytes", img.len())
}
Value::WalRecord(rec) => {
let wal_desc = pageserver_api::record::describe_wal_record(&rec)?;
let wal_desc = wal_decoder::models::record::describe_wal_record(&rec)?;
format!(
" rec {} bytes will_init: {} {}",
buf.len(),
@@ -1624,7 +1624,6 @@ pub(crate) mod test {
use bytes::Bytes;
use itertools::MinMaxResult;
use pageserver_api::value::Value;
use rand::prelude::{SeedableRng, SliceRandom, StdRng};
use rand::{Rng, RngCore};
@@ -1988,7 +1987,7 @@ pub(crate) mod test {
#[tokio::test]
async fn copy_delta_prefix_smoke() {
use bytes::Bytes;
use pageserver_api::record::NeonWalRecord;
use wal_decoder::models::record::NeonWalRecord;
let h = crate::tenant::harness::TenantHarness::create("truncate_delta_smoke")
.await

View File

@@ -4,8 +4,8 @@ use std::sync::Arc;
use anyhow::bail;
use pageserver_api::key::Key;
use pageserver_api::keyspace::{KeySpace, SparseKeySpace};
use pageserver_api::value::Value;
use utils::lsn::Lsn;
use wal_decoder::models::value::Value;
use super::PersistentLayerKey;
use super::merge_iterator::{MergeIterator, MergeIteratorItem};
@@ -126,7 +126,6 @@ mod tests {
#[tokio::test]
async fn filter_keyspace_iterator() {
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("filter_iterator_filter_keyspace_iterator")
.await

View File

@@ -42,7 +42,6 @@ use pageserver_api::config::MaxVectoredReadBytes;
use pageserver_api::key::{DBDIR_KEY, KEY_SIZE, Key};
use pageserver_api::keyspace::KeySpace;
use pageserver_api::shard::{ShardIdentity, TenantShardId};
use pageserver_api::value::Value;
use serde::{Deserialize, Serialize};
use tokio::sync::OnceCell;
use tokio_stream::StreamExt;
@@ -52,6 +51,7 @@ use utils::bin_ser::BeSer;
use utils::bin_ser::SerializeError;
use utils::id::{TenantId, TimelineId};
use utils::lsn::Lsn;
use wal_decoder::models::value::Value;
use super::errors::PutError;
use super::layer_name::ImageLayerName;
@@ -1232,10 +1232,10 @@ mod test {
use itertools::Itertools;
use pageserver_api::key::Key;
use pageserver_api::shard::{ShardCount, ShardIdentity, ShardNumber, ShardStripeSize};
use pageserver_api::value::Value;
use utils::generation::Generation;
use utils::id::{TenantId, TimelineId};
use utils::lsn::Lsn;
use wal_decoder::models::value::Value;
use super::{ImageLayerIterator, ImageLayerWriter};
use crate::DEFAULT_PG_VERSION;

View File

@@ -824,7 +824,7 @@ async fn evict_and_wait_does_not_wait_for_download() {
#[tokio::test(start_paused = true)]
async fn eviction_cancellation_on_drop() {
use bytes::Bytes;
use pageserver_api::value::Value;
use wal_decoder::models::value::Value;
// this is the runtime on which Layer spawns the blocking tasks on
let handle = tokio::runtime::Handle::current();

View File

@@ -4,8 +4,8 @@ use std::sync::Arc;
use anyhow::bail;
use pageserver_api::key::Key;
use pageserver_api::value::Value;
use utils::lsn::Lsn;
use wal_decoder::models::value::Value;
use super::delta_layer::{DeltaLayerInner, DeltaLayerIterator};
use super::image_layer::{ImageLayerInner, ImageLayerIterator};
@@ -402,9 +402,9 @@ impl<'a> MergeIterator<'a> {
mod tests {
use itertools::Itertools;
use pageserver_api::key::Key;
#[cfg(feature = "testing")]
use pageserver_api::record::NeonWalRecord;
use utils::lsn::Lsn;
#[cfg(feature = "testing")]
use wal_decoder::models::record::NeonWalRecord;
use super::*;
use crate::DEFAULT_PG_VERSION;
@@ -436,7 +436,6 @@ mod tests {
#[tokio::test]
async fn merge_in_between() {
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("merge_iterator_merge_in_between")
.await
@@ -501,7 +500,6 @@ mod tests {
#[tokio::test]
async fn delta_merge() {
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("merge_iterator_delta_merge")
.await
@@ -578,7 +576,6 @@ mod tests {
#[tokio::test]
async fn delta_image_mixed_merge() {
use bytes::Bytes;
use pageserver_api::value::Value;
let harness = TenantHarness::create("merge_iterator_delta_image_mixed_merge")
.await

View File

@@ -56,8 +56,6 @@ use pageserver_api::models::{
};
use pageserver_api::reltag::{BlockNumber, RelTag};
use pageserver_api::shard::{ShardIdentity, ShardIndex, ShardNumber, TenantShardId};
#[cfg(test)]
use pageserver_api::value::Value;
use postgres_connection::PgConnectionConfig;
use postgres_ffi::v14::xlog_utils;
use postgres_ffi::{WAL_SEGMENT_SIZE, to_pg_timestamp};
@@ -81,6 +79,8 @@ use utils::seqwait::SeqWait;
use utils::simple_rcu::{Rcu, RcuReadGuard};
use utils::sync::gate::{Gate, GateGuard};
use utils::{completion, critical, fs_ext, pausable_failpoint};
#[cfg(test)]
use wal_decoder::models::value::Value;
use wal_decoder::serialized_batch::{SerializedValueBatch, ValueMeta};
use self::delete::DeleteTimelineFlow;
@@ -5207,7 +5207,11 @@ impl Timeline {
}
let (dense_ks, sparse_ks) = self.collect_keyspace(lsn, ctx).await?;
let dense_partitioning = dense_ks.partition(&self.shard_identity, partition_size);
let dense_partitioning = dense_ks.partition(
&self.shard_identity,
partition_size,
postgres_ffi::BLCKSZ as u64,
);
let sparse_partitioning = SparseKeyPartitioning {
parts: vec![sparse_ks],
}; // no partitioning for metadata keys for now
@@ -7590,11 +7594,11 @@ mod tests {
use std::sync::Arc;
use pageserver_api::key::Key;
use pageserver_api::value::Value;
use std::iter::Iterator;
use tracing::Instrument;
use utils::id::TimelineId;
use utils::lsn::Lsn;
use wal_decoder::models::value::Value;
use super::HeatMapTimeline;
use crate::context::RequestContextBuilder;

View File

@@ -29,9 +29,7 @@ use pageserver_api::config::tenant_conf_defaults::DEFAULT_CHECKPOINT_DISTANCE;
use pageserver_api::key::{KEY_SIZE, Key};
use pageserver_api::keyspace::{KeySpace, ShardedRange};
use pageserver_api::models::{CompactInfoResponse, CompactKeyRange};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::{ShardCount, ShardIdentity, TenantShardId};
use pageserver_api::value::Value;
use pageserver_compaction::helpers::{fully_contains, overlaps_with};
use pageserver_compaction::interface::*;
use serde::Serialize;
@@ -41,6 +39,8 @@ use tracing::{Instrument, debug, error, info, info_span, trace, warn};
use utils::critical;
use utils::id::TimelineId;
use utils::lsn::Lsn;
use wal_decoder::models::record::NeonWalRecord;
use wal_decoder::models::value::Value;
use crate::context::{AccessStatsBehavior, RequestContext, RequestContextBuilder};
use crate::page_cache;

View File

@@ -36,8 +36,8 @@ use pageserver_api::keyspace::{ShardedRange, singleton_range};
use pageserver_api::models::{ShardImportProgress, ShardImportProgressV1, ShardImportStatus};
use pageserver_api::reltag::{RelTag, SlruKind};
use pageserver_api::shard::ShardIdentity;
use postgres_ffi::BLCKSZ;
use postgres_ffi::relfile_utils::parse_relfilename;
use postgres_ffi::{BLCKSZ, pg_constants};
use remote_storage::RemotePath;
use tokio::sync::Semaphore;
use tokio_stream::StreamExt;
@@ -558,7 +558,7 @@ impl PgDataDir {
PgDataDirDb::new(
storage,
&basedir.join(dboid.to_string()),
pg_constants::DEFAULTTABLESPACE_OID,
postgres_ffi_types::constants::DEFAULTTABLESPACE_OID,
dboid,
&datadir_path,
)
@@ -571,7 +571,7 @@ impl PgDataDir {
PgDataDirDb::new(
storage,
&datadir_path.join("global"),
postgres_ffi::pg_constants::GLOBALTABLESPACE_OID,
postgres_ffi_types::constants::GLOBALTABLESPACE_OID,
0,
&datadir_path,
)

View File

@@ -28,20 +28,20 @@ use std::time::{Duration, Instant, SystemTime};
use bytes::{Buf, Bytes};
use pageserver_api::key::{Key, rel_block_to_key};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::{BlockNumber, RelTag, SlruKind};
use pageserver_api::shard::ShardIdentity;
use postgres_ffi::relfile_utils::{FSM_FORKNUM, INIT_FORKNUM, MAIN_FORKNUM, VISIBILITYMAP_FORKNUM};
use postgres_ffi::walrecord::*;
use postgres_ffi::{
TimestampTz, TransactionId, dispatch_pgversion, enum_pgversion, enum_pgversion_dispatch,
fsm_logical_to_physical, pg_constants,
};
use postgres_ffi_types::forknum::{FSM_FORKNUM, INIT_FORKNUM, MAIN_FORKNUM, VISIBILITYMAP_FORKNUM};
use tracing::*;
use utils::bin_ser::{DeserializeError, SerializeError};
use utils::lsn::Lsn;
use utils::rate_limit::RateLimit;
use utils::{critical, failpoint_support};
use wal_decoder::models::record::NeonWalRecord;
use wal_decoder::models::*;
use crate::ZERO_PAGE;

View File

@@ -32,12 +32,12 @@ use anyhow::Context;
use bytes::{Bytes, BytesMut};
use pageserver_api::key::Key;
use pageserver_api::models::{WalRedoManagerProcessStatus, WalRedoManagerStatus};
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::TenantShardId;
use tracing::*;
use utils::lsn::Lsn;
use utils::sync::gate::GateError;
use utils::sync::heavier_once_cell;
use wal_decoder::models::record::NeonWalRecord;
use crate::config::PageServerConf;
use crate::metrics::{
@@ -571,11 +571,11 @@ mod tests {
use bytes::Bytes;
use pageserver_api::key::Key;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::shard::TenantShardId;
use tracing::Instrument;
use utils::id::TenantId;
use utils::lsn::Lsn;
use wal_decoder::models::record::NeonWalRecord;
use super::PostgresRedoManager;
use crate::config::PageServerConf;

View File

@@ -2,16 +2,16 @@ use anyhow::Context;
use byteorder::{ByteOrder, LittleEndian};
use bytes::BytesMut;
use pageserver_api::key::Key;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::SlruKind;
use postgres_ffi::relfile_utils::VISIBILITYMAP_FORKNUM;
use postgres_ffi::v14::nonrelfile_utils::{
mx_offset_to_flags_bitshift, mx_offset_to_flags_offset, mx_offset_to_member_offset,
transaction_id_set_status,
};
use postgres_ffi::{BLCKSZ, pg_constants};
use postgres_ffi_types::forknum::VISIBILITYMAP_FORKNUM;
use tracing::*;
use utils::lsn::Lsn;
use wal_decoder::models::record::NeonWalRecord;
/// Can this request be served by neon redo functions
/// or we need to pass it to wal-redo postgres process?

View File

@@ -10,7 +10,6 @@ use std::time::Duration;
use anyhow::Context;
use bytes::Bytes;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::reltag::RelTag;
use pageserver_api::shard::TenantShardId;
use postgres_ffi::BLCKSZ;
@@ -18,6 +17,7 @@ use tokio::io::{AsyncReadExt, AsyncWriteExt};
use tracing::{Instrument, debug, error, instrument};
use utils::lsn::Lsn;
use utils::poison::Poison;
use wal_decoder::models::record::NeonWalRecord;
use self::no_leak_child::NoLeakChild;
use crate::config::PageServerConf;