mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 13:32:57 +00:00
Extend replication protocol with ZentihFeedback message
to pass current_timeline_size to compute node Put standby_status_update fields into ZenithFeedback and send them as one message. Pass values sizes together with keys in ZenithFeedback message.
This commit is contained in:
@@ -20,8 +20,8 @@ clap = "2.33.0"
|
||||
daemonize = "0.4.1"
|
||||
rust-s3 = { version = "0.28", default-features = false, features = ["no-verify-ssl", "tokio-rustls-tls"] }
|
||||
tokio = { version = "1.11", features = ["macros"] }
|
||||
postgres-protocol = { git = "https://github.com/zenithdb/rust-postgres.git", rev="9eb0dbfbeb6a6c1b79099b9f7ae4a8c021877858" }
|
||||
postgres = { git = "https://github.com/zenithdb/rust-postgres.git", rev="9eb0dbfbeb6a6c1b79099b9f7ae4a8c021877858" }
|
||||
postgres-protocol = { git = "https://github.com/zenithdb/rust-postgres.git", rev="2949d98df52587d562986aad155dd4e889e408b7" }
|
||||
postgres = { git = "https://github.com/zenithdb/rust-postgres.git", rev="2949d98df52587d562986aad155dd4e889e408b7" }
|
||||
anyhow = "1.0"
|
||||
crc32c = "0.6.0"
|
||||
humantime = "2.1.0"
|
||||
@@ -30,7 +30,7 @@ signal-hook = "0.3.10"
|
||||
serde = { version = "1.0", features = ["derive"] }
|
||||
hex = "0.4.3"
|
||||
const_format = "0.2.21"
|
||||
tokio-postgres = { git = "https://github.com/zenithdb/rust-postgres.git", rev="9eb0dbfbeb6a6c1b79099b9f7ae4a8c021877858" }
|
||||
tokio-postgres = { git = "https://github.com/zenithdb/rust-postgres.git", rev="2949d98df52587d562986aad155dd4e889e408b7" }
|
||||
|
||||
postgres_ffi = { path = "../postgres_ffi" }
|
||||
workspace_hack = { path = "../workspace_hack" }
|
||||
|
||||
@@ -22,6 +22,7 @@ use zenith_metrics::{
|
||||
use zenith_utils::bin_ser::LeSer;
|
||||
use zenith_utils::lsn::Lsn;
|
||||
use zenith_utils::pq_proto::SystemId;
|
||||
use zenith_utils::pq_proto::ZenithFeedback;
|
||||
use zenith_utils::zid::{ZTenantId, ZTimelineId};
|
||||
|
||||
pub const SK_MAGIC: u32 = 0xcafeceefu32;
|
||||
@@ -278,9 +279,8 @@ pub struct AppendResponse {
|
||||
// We report back our awareness about which WAL is committed, as this is
|
||||
// a criterion for walproposer --sync mode exit
|
||||
pub commit_lsn: Lsn,
|
||||
// Min disk consistent lsn of pageservers (portion of WAL applied and written to the disk by pageservers)
|
||||
pub disk_consistent_lsn: Lsn,
|
||||
pub hs_feedback: HotStandbyFeedback,
|
||||
pub zenith_feedback: ZenithFeedback,
|
||||
}
|
||||
|
||||
impl AppendResponse {
|
||||
@@ -289,8 +289,8 @@ impl AppendResponse {
|
||||
term,
|
||||
flush_lsn: Lsn(0),
|
||||
commit_lsn: Lsn(0),
|
||||
disk_consistent_lsn: Lsn(0),
|
||||
hs_feedback: HotStandbyFeedback::empty(),
|
||||
zenith_feedback: ZenithFeedback::empty(),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -395,10 +395,11 @@ impl AcceptorProposerMessage {
|
||||
buf.put_u64_le(msg.term);
|
||||
buf.put_u64_le(msg.flush_lsn.into());
|
||||
buf.put_u64_le(msg.commit_lsn.into());
|
||||
buf.put_u64_le(msg.disk_consistent_lsn.into());
|
||||
buf.put_i64_le(msg.hs_feedback.ts);
|
||||
buf.put_u64_le(msg.hs_feedback.xmin);
|
||||
buf.put_u64_le(msg.hs_feedback.catalog_xmin);
|
||||
|
||||
msg.zenith_feedback.serialize(buf)?
|
||||
}
|
||||
}
|
||||
|
||||
@@ -614,9 +615,9 @@ where
|
||||
term: self.s.acceptor_state.term,
|
||||
flush_lsn: self.flush_lsn,
|
||||
commit_lsn: self.s.commit_lsn,
|
||||
disk_consistent_lsn: Lsn(0),
|
||||
// will be filled by the upper code to avoid bothering safekeeper
|
||||
hs_feedback: HotStandbyFeedback::empty(),
|
||||
zenith_feedback: ZenithFeedback::empty(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -9,6 +9,8 @@ use postgres_ffi::xlog_utils::{
|
||||
get_current_timestamp, TimestampTz, XLogFileName, MAX_SEND_SIZE, PG_TLI,
|
||||
};
|
||||
|
||||
use crate::callmemaybe::{CallmeEvent, SubscriptionStateKey};
|
||||
use bytes::Bytes;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use std::cmp::min;
|
||||
use std::fs::File;
|
||||
@@ -19,20 +21,21 @@ use std::sync::Arc;
|
||||
use std::thread::sleep;
|
||||
use std::time::Duration;
|
||||
use std::{str, thread};
|
||||
use tokio::sync::mpsc::UnboundedSender;
|
||||
use tracing::*;
|
||||
use zenith_utils::bin_ser::BeSer;
|
||||
use zenith_utils::lsn::Lsn;
|
||||
use zenith_utils::postgres_backend::PostgresBackend;
|
||||
use zenith_utils::pq_proto::{BeMessage, FeMessage, WalSndKeepAlive, XLogDataBody};
|
||||
use zenith_utils::pq_proto::{BeMessage, FeMessage, WalSndKeepAlive, XLogDataBody, ZenithFeedback};
|
||||
use zenith_utils::sock_split::ReadStream;
|
||||
|
||||
use crate::callmemaybe::{CallmeEvent, SubscriptionStateKey};
|
||||
use tokio::sync::mpsc::UnboundedSender;
|
||||
use zenith_utils::zid::{ZTenantId, ZTimelineId};
|
||||
|
||||
// See: https://www.postgresql.org/docs/13/protocol-replication.html
|
||||
const HOT_STANDBY_FEEDBACK_TAG_BYTE: u8 = b'h';
|
||||
const STANDBY_STATUS_UPDATE_TAG_BYTE: u8 = b'r';
|
||||
// zenith extension of replication protocol
|
||||
const ZENITH_STATUS_UPDATE_TAG_BYTE: u8 = b'z';
|
||||
|
||||
type FullTransactionId = u64;
|
||||
|
||||
@@ -139,8 +142,8 @@ impl ReplicationConn {
|
||||
while let Some(msg) = FeMessage::read(&mut stream_in)? {
|
||||
match &msg {
|
||||
FeMessage::CopyData(m) => {
|
||||
// There's two possible data messages that the client is supposed to send here:
|
||||
// `HotStandbyFeedback` and `StandbyStatusUpdate`.
|
||||
// There's three possible data messages that the client is supposed to send here:
|
||||
// `HotStandbyFeedback` and `StandbyStatusUpdate` and `ZenithStandbyFeedback`.
|
||||
|
||||
match m.first().cloned() {
|
||||
Some(HOT_STANDBY_FEEDBACK_TAG_BYTE) => {
|
||||
@@ -150,11 +153,25 @@ impl ReplicationConn {
|
||||
timeline.update_replica_state(replica_id, state);
|
||||
}
|
||||
Some(STANDBY_STATUS_UPDATE_TAG_BYTE) => {
|
||||
let reply = StandbyReply::des(&m[1..])
|
||||
let _reply = StandbyReply::des(&m[1..])
|
||||
.context("failed to deserialize StandbyReply")?;
|
||||
state.last_received_lsn = reply.write_lsn;
|
||||
state.disk_consistent_lsn = reply.flush_lsn;
|
||||
state.remote_consistent_lsn = reply.apply_lsn;
|
||||
// This must be a regular postgres replica,
|
||||
// because pageserver doesn't send this type of messages to safekeeper.
|
||||
// Currently this is not implemented, so this message is ignored.
|
||||
|
||||
warn!("unexpected StandbyReply. Read-only postgres replicas are not supported in safekeepers yet.");
|
||||
// timeline.update_replica_state(replica_id, Some(state));
|
||||
}
|
||||
Some(ZENITH_STATUS_UPDATE_TAG_BYTE) => {
|
||||
// Note: deserializing is on m[9..] because we skip the tag byte and len bytes.
|
||||
let buf = Bytes::copy_from_slice(&m[9..]);
|
||||
let reply = ZenithFeedback::parse(buf);
|
||||
|
||||
info!("ZenithFeedback is {:?}", reply);
|
||||
// Only pageserver sends ZenithFeedback, so set the flag.
|
||||
// This replica is the source of information to resend to compute.
|
||||
state.zenith_feedback = Some(reply);
|
||||
|
||||
timeline.update_replica_state(replica_id, state);
|
||||
}
|
||||
_ => warn!("unexpected message {:?}", msg),
|
||||
|
||||
@@ -27,6 +27,7 @@ use crate::upgrade::upgrade_control_file;
|
||||
use crate::SafeKeeperConf;
|
||||
use postgres_ffi::xlog_utils::{XLogFileName, XLOG_BLCKSZ};
|
||||
use std::convert::TryInto;
|
||||
use zenith_utils::pq_proto::ZenithFeedback;
|
||||
|
||||
// contains persistent metadata for safekeeper
|
||||
const CONTROL_FILE_NAME: &str = "safekeeper.control";
|
||||
@@ -35,17 +36,17 @@ const CONTROL_FILE_NAME_PARTIAL: &str = "safekeeper.control.partial";
|
||||
const POLL_STATE_TIMEOUT: Duration = Duration::from_secs(1);
|
||||
pub const CHECKSUM_SIZE: usize = std::mem::size_of::<u32>();
|
||||
|
||||
/// Replica status: host standby feedback + disk consistent lsn
|
||||
/// Replica status update + hot standby feedback
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub struct ReplicaState {
|
||||
/// last known lsn received by replica
|
||||
pub last_received_lsn: Lsn, // None means we don't know
|
||||
/// combined disk_consistent_lsn of pageservers
|
||||
pub disk_consistent_lsn: Lsn,
|
||||
/// combined remote consistent lsn of pageservers
|
||||
pub remote_consistent_lsn: Lsn,
|
||||
/// combined hot standby feedback from all replicas
|
||||
pub hs_feedback: HotStandbyFeedback,
|
||||
/// Zenith specific feedback received from pageserver, if any
|
||||
pub zenith_feedback: Option<ZenithFeedback>,
|
||||
}
|
||||
|
||||
impl Default for ReplicaState {
|
||||
@@ -58,13 +59,13 @@ impl ReplicaState {
|
||||
pub fn new() -> ReplicaState {
|
||||
ReplicaState {
|
||||
last_received_lsn: Lsn::MAX,
|
||||
disk_consistent_lsn: Lsn(u64::MAX),
|
||||
remote_consistent_lsn: Lsn(u64::MAX),
|
||||
remote_consistent_lsn: Lsn(0),
|
||||
hs_feedback: HotStandbyFeedback {
|
||||
ts: 0,
|
||||
xmin: u64::MAX,
|
||||
catalog_xmin: u64::MAX,
|
||||
},
|
||||
zenith_feedback: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -109,13 +110,40 @@ impl SharedState {
|
||||
acc.hs_feedback.xmin = min(acc.hs_feedback.xmin, state.hs_feedback.xmin);
|
||||
acc.hs_feedback.catalog_xmin =
|
||||
min(acc.hs_feedback.catalog_xmin, state.hs_feedback.catalog_xmin);
|
||||
acc.disk_consistent_lsn = Lsn::min(acc.disk_consistent_lsn, state.disk_consistent_lsn);
|
||||
// currently not used, but update it to be consistent
|
||||
acc.last_received_lsn = Lsn::min(acc.last_received_lsn, state.last_received_lsn);
|
||||
// When at least one replica has preserved data up to remote_consistent_lsn,
|
||||
// safekeeper is free to delete it, so chose max of all replicas.
|
||||
acc.remote_consistent_lsn =
|
||||
Lsn::max(acc.remote_consistent_lsn, state.remote_consistent_lsn);
|
||||
|
||||
// FIXME
|
||||
// If multiple pageservers are streaming WAL and send feedback for the same timeline simultaneously,
|
||||
// this code is not correct.
|
||||
// Now the most advanced feedback is used.
|
||||
// If one pageserver lags when another doesn't, the backpressure won't be activated on compute and lagging
|
||||
// pageserver is prone to timeout errors.
|
||||
//
|
||||
// To choose what feedback to use and resend to compute node,
|
||||
// we need to know which pageserver compute node considers to be main.
|
||||
// See https://github.com/zenithdb/zenith/issues/1171
|
||||
//
|
||||
if let Some(zenith_feedback) = state.zenith_feedback {
|
||||
if let Some(acc_feedback) = acc.zenith_feedback {
|
||||
if acc_feedback.ps_writelsn < zenith_feedback.ps_writelsn {
|
||||
warn!("More than one pageserver is streaming WAL for the timeline. Feedback resolving is not fully supported yet.");
|
||||
acc.zenith_feedback = Some(zenith_feedback);
|
||||
}
|
||||
} else {
|
||||
acc.zenith_feedback = Some(zenith_feedback);
|
||||
}
|
||||
|
||||
// last lsn received by pageserver
|
||||
// FIXME if multiple pageservers are streaming WAL, last_received_lsn must be tracked per pageserver.
|
||||
// See https://github.com/zenithdb/zenith/issues/1171
|
||||
acc.last_received_lsn = Lsn::from(zenith_feedback.ps_writelsn);
|
||||
|
||||
// When at least one pageserver has preserved data up to remote_consistent_lsn,
|
||||
// safekeeper is free to delete it, so choose max of all pageservers.
|
||||
acc.remote_consistent_lsn = max(
|
||||
Lsn::from(zenith_feedback.ps_applylsn),
|
||||
acc.remote_consistent_lsn,
|
||||
);
|
||||
}
|
||||
}
|
||||
acc
|
||||
}
|
||||
@@ -280,8 +308,9 @@ impl Timeline {
|
||||
if let Some(AcceptorProposerMessage::AppendResponse(ref mut resp)) = rmsg {
|
||||
let state = shared_state.get_replicas_state();
|
||||
resp.hs_feedback = state.hs_feedback;
|
||||
resp.disk_consistent_lsn = state.disk_consistent_lsn;
|
||||
// XXX Do we need to add state.last_received_lsn to resp?
|
||||
if let Some(zenith_feedback) = state.zenith_feedback {
|
||||
resp.zenith_feedback = zenith_feedback;
|
||||
}
|
||||
}
|
||||
}
|
||||
// Ping wal sender that new data might be available.
|
||||
|
||||
Reference in New Issue
Block a user