Use enum-typed PG versions (#12317)

This makes it possible for the compiler to validate that a match block
matched all PostgreSQL versions we support.

## Problem
We did not have a complete picture about which places we had to test
against PG versions, and what format these versions were: The full PG
version ID format (Major/minor/bugfix `MMmmbb`) as transfered in
protocol messages, or only the Major release version (`MM`). This meant
type confusion was rampant.

With this change, it becomes easier to develop new version-dependent
features, by making type and niche confusion impossible.

## Summary of changes
Every use of `pg_version` is now typed as either `PgVersionId` (u32,
valued in decimal `MMmmbb`) or PgMajorVersion (an enum, with a value for
every major version we support, serialized and stored like a u32 with
the value of that major version)

---------

Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
This commit is contained in:
Matthias van de Meent
2025-06-24 19:25:31 +02:00
committed by GitHub
parent 158d84ea30
commit 6c6de6382a
62 changed files with 683 additions and 386 deletions

View File

@@ -58,6 +58,7 @@ metrics.workspace = true
pem.workspace = true
postgres_backend.workspace = true
postgres_ffi.workspace = true
postgres_versioninfo.workspace = true
pq_proto.workspace = true
remote_storage.workspace = true
safekeeper_api.workspace = true

View File

@@ -2,6 +2,7 @@
use std::vec;
use anyhow::{Result, bail};
use postgres_versioninfo::PgVersionId;
use pq_proto::SystemId;
use safekeeper_api::membership::{Configuration, INVALID_GENERATION};
use safekeeper_api::{ServerInfo, Term};
@@ -46,7 +47,7 @@ struct SafeKeeperStateV1 {
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct ServerInfoV2 {
/// Postgres server version
pub pg_version: u32,
pub pg_version: PgVersionId,
pub system_id: SystemId,
pub tenant_id: TenantId,
pub timeline_id: TimelineId,
@@ -75,7 +76,7 @@ pub struct SafeKeeperStateV2 {
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct ServerInfoV3 {
/// Postgres server version
pub pg_version: u32,
pub pg_version: PgVersionId,
pub system_id: SystemId,
#[serde(with = "hex")]
pub tenant_id: TenantId,
@@ -444,13 +445,13 @@ pub fn upgrade_control_file(buf: &[u8], version: u32) -> Result<TimelinePersiste
} else if version == 6 {
info!("reading safekeeper control file version {}", version);
let mut oldstate = TimelinePersistentState::des(&buf[..buf.len()])?;
if oldstate.server.pg_version != 0 {
if oldstate.server.pg_version != PgVersionId::UNKNOWN {
return Ok(oldstate);
}
// set pg_version to the default v14
info!("setting pg_version to 140005");
oldstate.server.pg_version = 140005;
oldstate.server.pg_version = PgVersionId::from_full_pg_version(140005);
return Ok(oldstate);
} else if version == 7 {
@@ -547,6 +548,7 @@ pub fn downgrade_v10_to_v9(state: &TimelinePersistentState) -> TimelinePersisten
mod tests {
use std::str::FromStr;
use postgres_versioninfo::PgMajorVersion;
use utils::Hex;
use utils::id::NodeId;
@@ -563,7 +565,7 @@ mod tests {
epoch: 43,
},
server: ServerInfoV2 {
pg_version: 14,
pg_version: PgVersionId::from(PgMajorVersion::PG14),
system_id: 0x1234567887654321,
tenant_id,
timeline_id,
@@ -586,8 +588,8 @@ mod tests {
0x2a, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
// epoch
0x2b, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
// pg_version
0x0e, 0x00, 0x00, 0x00,
// pg_version = 140000
0xE0, 0x22, 0x02, 0x00,
// system_id
0x21, 0x43, 0x65, 0x87, 0x78, 0x56, 0x34, 0x12,
// tenant_id
@@ -626,7 +628,7 @@ mod tests {
}]),
},
server: ServerInfoV2 {
pg_version: 14,
pg_version: PgVersionId::from(PgMajorVersion::PG14),
system_id: 0x1234567887654321,
tenant_id,
timeline_id,
@@ -646,7 +648,7 @@ mod tests {
let expected = [
0x2a, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x29, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x0e, 0x00, 0x00, 0x00, 0x21, 0x43, 0x65, 0x87, 0x78, 0x56,
0x00, 0x00, 0x00, 0x00, 0xE0, 0x22, 0x02, 0x00, 0x21, 0x43, 0x65, 0x87, 0x78, 0x56,
0x34, 0x12, 0xcf, 0x04, 0x80, 0x92, 0x97, 0x07, 0xee, 0x75, 0x37, 0x23, 0x37, 0xef,
0xaa, 0x5e, 0xcf, 0x96, 0x11, 0x2d, 0xed, 0x66, 0x42, 0x2a, 0xa5, 0xe9, 0x53, 0xe5,
0x44, 0x0f, 0xa5, 0x42, 0x7a, 0xc4, 0x78, 0x56, 0x34, 0x12, 0xc4, 0x7a, 0x42, 0xa5,
@@ -675,7 +677,7 @@ mod tests {
}]),
},
server: ServerInfoV3 {
pg_version: 14,
pg_version: PgVersionId::from(PgMajorVersion::PG14),
system_id: 0x1234567887654321,
tenant_id,
timeline_id,
@@ -695,7 +697,7 @@ mod tests {
let expected = [
0x2a, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x29, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x0e, 0x00, 0x00, 0x00, 0x21, 0x43, 0x65, 0x87, 0x78, 0x56,
0x00, 0x00, 0x00, 0x00, 0xE0, 0x22, 0x02, 0x00, 0x21, 0x43, 0x65, 0x87, 0x78, 0x56,
0x34, 0x12, 0x20, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x63, 0x66, 0x30, 0x34,
0x38, 0x30, 0x39, 0x32, 0x39, 0x37, 0x30, 0x37, 0x65, 0x65, 0x37, 0x35, 0x33, 0x37,
0x32, 0x33, 0x33, 0x37, 0x65, 0x66, 0x61, 0x61, 0x35, 0x65, 0x63, 0x66, 0x39, 0x36,
@@ -731,7 +733,7 @@ mod tests {
}]),
},
server: ServerInfo {
pg_version: 14,
pg_version: PgVersionId::from(PgMajorVersion::PG14),
system_id: 0x1234567887654321,
wal_seg_size: 0x12345678,
},
@@ -765,7 +767,7 @@ mod tests {
0x30, 0x66, 0x61, 0x35, 0x34, 0x32, 0x37, 0x61, 0x63, 0x34, 0x2a, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x29, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x0e, 0x00, 0x00, 0x00, 0x21, 0x43, 0x65, 0x87, 0x78, 0x56, 0x34, 0x12, 0x78, 0x56,
0xE0, 0x22, 0x02, 0x00, 0x21, 0x43, 0x65, 0x87, 0x78, 0x56, 0x34, 0x12, 0x78, 0x56,
0x34, 0x12, 0x20, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x63, 0x34, 0x37, 0x61,
0x34, 0x32, 0x61, 0x35, 0x30, 0x66, 0x34, 0x34, 0x65, 0x35, 0x35, 0x33, 0x65, 0x39,
0x61, 0x35, 0x32, 0x61, 0x34, 0x32, 0x36, 0x36, 0x65, 0x64, 0x32, 0x64, 0x31, 0x31,

View File

@@ -9,6 +9,7 @@ use anyhow::{Context, Result, bail};
use byteorder::{LittleEndian, ReadBytesExt};
use bytes::{Buf, BufMut, Bytes, BytesMut};
use postgres_ffi::{MAX_SEND_SIZE, TimeLineID};
use postgres_versioninfo::{PgMajorVersion, PgVersionId};
use pq_proto::SystemId;
use safekeeper_api::membership::{
INVALID_GENERATION, MemberSet, SafekeeperGeneration as Generation, SafekeeperId,
@@ -29,7 +30,7 @@ use crate::{control_file, wal_storage};
pub const SK_PROTO_VERSION_2: u32 = 2;
pub const SK_PROTO_VERSION_3: u32 = 3;
pub const UNKNOWN_SERVER_VERSION: u32 = 0;
pub const UNKNOWN_SERVER_VERSION: PgVersionId = PgVersionId::UNKNOWN;
#[derive(Debug, Clone, Copy, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)]
pub struct TermLsn {
@@ -218,7 +219,7 @@ pub struct ProposerGreeting {
pub timeline_id: TimelineId,
pub mconf: membership::Configuration,
/// Postgres server version
pub pg_version: u32,
pub pg_version: PgVersionId,
pub system_id: SystemId,
pub wal_seg_size: u32,
}
@@ -229,7 +230,7 @@ pub struct ProposerGreetingV2 {
/// proposer-acceptor protocol version
pub protocol_version: u32,
/// Postgres server version
pub pg_version: u32,
pub pg_version: PgVersionId,
pub proposer_id: PgUuid,
pub system_id: SystemId,
pub timeline_id: TimelineId,
@@ -511,7 +512,7 @@ impl ProposerAcceptorMessage {
tenant_id,
timeline_id,
mconf,
pg_version,
pg_version: PgVersionId::from_full_pg_version(pg_version),
system_id,
wal_seg_size,
};
@@ -961,7 +962,8 @@ where
* because safekeepers parse WAL headers and the format
* may change between versions.
*/
if msg.pg_version / 10000 != self.state.server.pg_version / 10000
if PgMajorVersion::try_from(msg.pg_version)?
!= PgMajorVersion::try_from(self.state.server.pg_version)?
&& self.state.server.pg_version != UNKNOWN_SERVER_VERSION
{
bail!(
@@ -1748,7 +1750,7 @@ mod tests {
}]),
},
server: ServerInfo {
pg_version: 14,
pg_version: PgVersionId::from_full_pg_version(140000),
system_id: 0x1234567887654321,
wal_seg_size: 0x12345678,
},

View File

@@ -8,8 +8,8 @@ use futures::StreamExt;
use futures::future::Either;
use pageserver_api::shard::ShardIdentity;
use postgres_backend::{CopyStreamHandlerEnd, PostgresBackend};
use postgres_ffi::get_current_timestamp;
use postgres_ffi::waldecoder::{WalDecodeError, WalStreamDecoder};
use postgres_ffi::{PgMajorVersion, get_current_timestamp};
use pq_proto::{BeMessage, InterpretedWalRecordsBody, WalSndKeepAlive};
use tokio::io::{AsyncRead, AsyncWrite};
use tokio::sync::mpsc::error::SendError;
@@ -78,7 +78,7 @@ pub(crate) struct InterpretedWalReader {
shard_senders: HashMap<ShardIdentity, smallvec::SmallVec<[ShardSenderState; 1]>>,
shard_notification_rx: Option<tokio::sync::mpsc::UnboundedReceiver<AttachShardNotification>>,
state: Arc<std::sync::RwLock<InterpretedWalReaderState>>,
pg_version: u32,
pg_version: PgMajorVersion,
}
/// A handle for [`InterpretedWalReader`] which allows for interacting with it
@@ -258,7 +258,7 @@ impl InterpretedWalReader {
start_pos: Lsn,
tx: tokio::sync::mpsc::Sender<Batch>,
shard: ShardIdentity,
pg_version: u32,
pg_version: PgMajorVersion,
appname: &Option<String>,
) -> InterpretedWalReaderHandle {
let state = Arc::new(std::sync::RwLock::new(InterpretedWalReaderState::Running {
@@ -322,7 +322,7 @@ impl InterpretedWalReader {
start_pos: Lsn,
tx: tokio::sync::mpsc::Sender<Batch>,
shard: ShardIdentity,
pg_version: u32,
pg_version: PgMajorVersion,
shard_notification_rx: Option<
tokio::sync::mpsc::UnboundedReceiver<AttachShardNotification>,
>,
@@ -718,7 +718,7 @@ mod tests {
use std::time::Duration;
use pageserver_api::shard::{ShardIdentity, ShardStripeSize};
use postgres_ffi::MAX_SEND_SIZE;
use postgres_ffi::{MAX_SEND_SIZE, PgMajorVersion};
use tokio::sync::mpsc::error::TryRecvError;
use utils::id::{NodeId, TenantTimelineId};
use utils::lsn::Lsn;
@@ -734,7 +734,7 @@ mod tests {
const SIZE: usize = 8 * 1024;
const MSG_COUNT: usize = 200;
const PG_VERSION: u32 = 17;
const PG_VERSION: PgMajorVersion = PgMajorVersion::PG17;
const SHARD_COUNT: u8 = 2;
let start_lsn = Lsn::from_str("0/149FD18").unwrap();
@@ -876,7 +876,7 @@ mod tests {
const SIZE: usize = 8 * 1024;
const MSG_COUNT: usize = 200;
const PG_VERSION: u32 = 17;
const PG_VERSION: PgMajorVersion = PgMajorVersion::PG17;
const SHARD_COUNT: u8 = 2;
let start_lsn = Lsn::from_str("0/149FD18").unwrap();
@@ -1025,7 +1025,7 @@ mod tests {
const SIZE: usize = 64 * 1024;
const MSG_COUNT: usize = 10;
const PG_VERSION: u32 = 17;
const PG_VERSION: PgMajorVersion = PgMajorVersion::PG17;
const SHARD_COUNT: u8 = 2;
const WAL_READER_BATCH_SIZE: usize = 8192;
@@ -1148,7 +1148,7 @@ mod tests {
const SIZE: usize = 8 * 1024;
const MSG_COUNT: usize = 10;
const PG_VERSION: u32 = 17;
const PG_VERSION: PgMajorVersion = PgMajorVersion::PG17;
let start_lsn = Lsn::from_str("0/149FD18").unwrap();
let env = Env::new(true).unwrap();

View File

@@ -12,7 +12,7 @@ use futures::FutureExt;
use itertools::Itertools;
use parking_lot::Mutex;
use postgres_backend::{CopyStreamHandlerEnd, PostgresBackend, PostgresBackendReader, QueryError};
use postgres_ffi::{MAX_SEND_SIZE, TimestampTz, get_current_timestamp};
use postgres_ffi::{MAX_SEND_SIZE, PgMajorVersion, TimestampTz, get_current_timestamp};
use pq_proto::{BeMessage, WalSndKeepAlive, XLogDataBody};
use safekeeper_api::Term;
use safekeeper_api::models::{
@@ -559,7 +559,9 @@ impl SafekeeperPostgresHandler {
format,
compression,
} => {
let pg_version = tli.tli.get_state().await.1.server.pg_version / 10000;
let pg_version =
PgMajorVersion::try_from(tli.tli.get_state().await.1.server.pg_version)
.unwrap();
let end_watch_view = end_watch.view();
let wal_residence_guard = tli.wal_residence_guard().await?;
let (tx, rx) = tokio::sync::mpsc::channel::<Batch>(2);

View File

@@ -7,6 +7,7 @@ use std::time::SystemTime;
use anyhow::{Result, bail};
use postgres_ffi::WAL_SEGMENT_SIZE;
use postgres_versioninfo::{PgMajorVersion, PgVersionId};
use safekeeper_api::membership::Configuration;
use safekeeper_api::models::{TimelineMembershipSwitchResponse, TimelineTermBumpResponse};
use safekeeper_api::{INITIAL_TERM, ServerInfo, Term};
@@ -149,8 +150,8 @@ impl TimelinePersistentState {
&TenantTimelineId::empty(),
Configuration::empty(),
ServerInfo {
pg_version: 170000, /* Postgres server version (major * 10000) */
system_id: 0, /* Postgres system identifier */
pg_version: PgVersionId::from(PgMajorVersion::PG17),
system_id: 0, /* Postgres system identifier */
wal_seg_size: WAL_SEGMENT_SIZE as u32,
},
Lsn::INVALID,

View File

@@ -19,6 +19,7 @@ use futures::future::BoxFuture;
use postgres_ffi::v14::xlog_utils::{IsPartialXLogFileName, IsXLogFileName, XLogFromFileName};
use postgres_ffi::waldecoder::WalStreamDecoder;
use postgres_ffi::{PG_TLI, XLogFileName, XLogSegNo, dispatch_pgversion};
use postgres_versioninfo::{PgMajorVersion, PgVersionId};
use pq_proto::SystemId;
use remote_storage::RemotePath;
use std::sync::Arc;
@@ -92,7 +93,7 @@ pub struct PhysicalStorage {
/// Size of WAL segment in bytes.
wal_seg_size: usize,
pg_version: u32,
pg_version: PgVersionId,
system_id: u64,
/// Written to disk, but possibly still in the cache and not fully persisted.
@@ -180,7 +181,7 @@ impl PhysicalStorage {
let write_lsn = if state.commit_lsn == Lsn(0) {
Lsn(0)
} else {
let version = state.server.pg_version / 10000;
let version = PgMajorVersion::try_from(state.server.pg_version).unwrap();
dispatch_pgversion!(
version,
@@ -226,7 +227,10 @@ impl PhysicalStorage {
write_record_lsn: write_lsn,
flush_lsn,
flush_record_lsn: flush_lsn,
decoder: WalStreamDecoder::new(write_lsn, state.server.pg_version / 10000),
decoder: WalStreamDecoder::new(
write_lsn,
PgMajorVersion::try_from(state.server.pg_version).unwrap(),
),
file: None,
pending_wal_truncation: true,
})
@@ -408,7 +412,7 @@ impl Storage for PhysicalStorage {
let segno = init_lsn.segment_number(self.wal_seg_size);
let (mut file, _) = self.open_or_create(segno).await?;
let major_pg_version = self.pg_version / 10000;
let major_pg_version = PgMajorVersion::try_from(self.pg_version).unwrap();
let wal_seg =
postgres_ffi::generate_wal_segment(segno, self.system_id, major_pg_version, init_lsn)?;
file.seek(SeekFrom::Start(0)).await?;
@@ -654,7 +658,7 @@ pub struct WalReader {
// pos is in the same segment as timeline_start_lsn.
timeline_start_lsn: Lsn,
// integer version number of PostgreSQL, e.g. 14; 15; 16
pg_version: u32,
pg_version: PgMajorVersion,
system_id: SystemId,
timeline_start_segment: Option<Bytes>,
}
@@ -697,7 +701,7 @@ impl WalReader {
wal_backup,
local_start_lsn: state.local_start_lsn,
timeline_start_lsn: state.timeline_start_lsn,
pg_version: state.server.pg_version / 10000,
pg_version: PgMajorVersion::try_from(state.server.pg_version).unwrap(),
system_id: state.server.system_id,
timeline_start_segment: None,
})

View File

@@ -7,8 +7,8 @@ use anyhow::Result;
use bytes::{Buf, BytesMut};
use futures::future::BoxFuture;
use parking_lot::Mutex;
use postgres_ffi::XLogSegNo;
use postgres_ffi::waldecoder::WalStreamDecoder;
use postgres_ffi::{PgMajorVersion, XLogSegNo};
use safekeeper::metrics::WalStorageMetrics;
use safekeeper::state::TimelinePersistentState;
use safekeeper::{control_file, wal_storage};
@@ -142,7 +142,7 @@ impl DiskWALStorage {
write_lsn,
write_record_lsn: flush_lsn,
flush_record_lsn: flush_lsn,
decoder: WalStreamDecoder::new(flush_lsn, 16),
decoder: WalStreamDecoder::new(flush_lsn, PgMajorVersion::PG16),
unflushed_bytes: BytesMut::new(),
disk,
})
@@ -151,7 +151,7 @@ impl DiskWALStorage {
fn find_end_of_wal(disk: Arc<TimelineDisk>, start_lsn: Lsn) -> Result<Lsn> {
let mut buf = [0; 8192];
let mut pos = start_lsn.0;
let mut decoder = WalStreamDecoder::new(start_lsn, 16);
let mut decoder = WalStreamDecoder::new(start_lsn, PgMajorVersion::PG16);
let mut result = start_lsn;
loop {
disk.wal.lock().read(pos, &mut buf);
@@ -204,7 +204,7 @@ impl wal_storage::Storage for DiskWALStorage {
self.decoder.available(),
startpos,
);
self.decoder = WalStreamDecoder::new(startpos, 16);
self.decoder = WalStreamDecoder::new(startpos, PgMajorVersion::PG16);
}
self.decoder.feed_bytes(buf);
loop {
@@ -242,7 +242,7 @@ impl wal_storage::Storage for DiskWALStorage {
self.write_record_lsn = end_pos;
self.flush_record_lsn = end_pos;
self.unflushed_bytes.clear();
self.decoder = WalStreamDecoder::new(end_pos, 16);
self.decoder = WalStreamDecoder::new(end_pos, PgMajorVersion::PG16);
Ok(())
}