mirror of
https://github.com/neondatabase/neon.git
synced 2025-12-22 21:59:59 +00:00
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:
committed by
GitHub
parent
158d84ea30
commit
6c6de6382a
@@ -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
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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,
|
||||
},
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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,
|
||||
})
|
||||
|
||||
@@ -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(())
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user