Files
neon/safekeeper/src/handler.rs
Arthur Petukhovsky 976576ae59 Fix walreceiver and safekeeper bugs (#2295)
- There was an issue with zero commit_lsn `reason: LaggingWal { current_commit_lsn: 0/0, new_commit_lsn: 1/6FD90D38, threshold: 10485760 } }`. The problem was in `send_wal.rs`, where we initialized `end_pos = Lsn(0)` and in some cases sent it to the pageserver.
- IDENTIFY_SYSTEM previously returned `flush_lsn` as a physical end of WAL. Now it returns `flush_lsn` (as it was) to walproposer and `commit_lsn` to everyone else including pageserver.
- There was an issue with backoff where connection was cancelled right after initialization: `connected!` -> `safekeeper_handle_db: Connection cancelled` -> `Backoff: waiting 3 seconds`. The problem was in sleeping before establishing the connection. This is fixed by reworking retry logic.
- There was an issue with getting `NoKeepAlives` reason in a loop. The issue is probably the same as the previous.
- There was an issue with filtering safekeepers based on retry attempts, which could filter some safekeepers indefinetely. This is fixed by using retry cooldown duration instead of retry attempts.
- Some `send_wal.rs` connections failed with errors without context. This is fixed by adding a timeline to safekeepers errors.

New retry logic works like this:
- Every candidate has a `next_retry_at` timestamp and is not considered for connection until that moment
- When walreceiver connection is closed, we update `next_retry_at` using exponential backoff, increasing the cooldown on every disconnect.
- When `last_record_lsn` was advanced using the WAL from the safekeeper, we reset the retry cooldown and exponential backoff, allowing walreceiver to reconnect to the same safekeeper instantly.
2022-08-18 13:38:23 +03:00

218 lines
7.3 KiB
Rust

//! Part of Safekeeper pretending to be Postgres, i.e. handling Postgres
//! protocol commands.
use crate::json_ctrl::{handle_json_ctrl, AppendLogicalMessage};
use crate::receive_wal::ReceiveWalConn;
use crate::safekeeper::{AcceptorProposerMessage, ProposerAcceptorMessage};
use crate::send_wal::ReplicationConn;
use crate::timeline::{Timeline, TimelineTools};
use crate::SafeKeeperConf;
use anyhow::{bail, Context, Result};
use postgres_ffi::PG_TLI;
use regex::Regex;
use std::str::FromStr;
use std::sync::Arc;
use tracing::info;
use utils::{
lsn::Lsn,
postgres_backend::{self, PostgresBackend},
pq_proto::{BeMessage, FeStartupPacket, RowDescriptor, INT4_OID, TEXT_OID},
zid::{ZTenantId, ZTenantTimelineId, ZTimelineId},
};
/// Safekeeper handler of postgres commands
pub struct SafekeeperPostgresHandler {
pub conf: SafeKeeperConf,
/// assigned application name
pub appname: Option<String>,
pub ztenantid: Option<ZTenantId>,
pub ztimelineid: Option<ZTimelineId>,
pub timeline: Option<Arc<Timeline>>,
}
/// Parsed Postgres command.
enum SafekeeperPostgresCommand {
StartWalPush,
StartReplication { start_lsn: Lsn },
IdentifySystem,
JSONCtrl { cmd: AppendLogicalMessage },
}
fn parse_cmd(cmd: &str) -> Result<SafekeeperPostgresCommand> {
if cmd.starts_with("START_WAL_PUSH") {
Ok(SafekeeperPostgresCommand::StartWalPush)
} else if cmd.starts_with("START_REPLICATION") {
let re =
Regex::new(r"START_REPLICATION(?: PHYSICAL)? ([[:xdigit:]]+/[[:xdigit:]]+)").unwrap();
let mut caps = re.captures_iter(cmd);
let start_lsn = caps
.next()
.map(|cap| cap[1].parse::<Lsn>())
.context("failed to parse start LSN from START_REPLICATION command")??;
Ok(SafekeeperPostgresCommand::StartReplication { start_lsn })
} else if cmd.starts_with("IDENTIFY_SYSTEM") {
Ok(SafekeeperPostgresCommand::IdentifySystem)
} else if cmd.starts_with("JSON_CTRL") {
let cmd = cmd.strip_prefix("JSON_CTRL").context("invalid prefix")?;
Ok(SafekeeperPostgresCommand::JSONCtrl {
cmd: serde_json::from_str(cmd)?,
})
} else {
bail!("unsupported command {}", cmd);
}
}
impl postgres_backend::Handler for SafekeeperPostgresHandler {
// ztenant id and ztimeline id are passed in connection string params
fn startup(&mut self, _pgb: &mut PostgresBackend, sm: &FeStartupPacket) -> Result<()> {
if let FeStartupPacket::StartupMessage { params, .. } = sm {
self.ztenantid = match params.get("ztenantid") {
Some(z) => Some(ZTenantId::from_str(z)?), // just curious, can I do that from .map?
_ => None,
};
self.ztimelineid = match params.get("ztimelineid") {
Some(z) => Some(ZTimelineId::from_str(z)?),
_ => None,
};
if let Some(app_name) = params.get("application_name") {
self.appname = Some(app_name.clone());
}
Ok(())
} else {
bail!("Safekeeper received unexpected initial message: {:?}", sm);
}
}
fn process_query(&mut self, pgb: &mut PostgresBackend, query_string: &str) -> Result<()> {
let cmd = parse_cmd(query_string)?;
info!(
"got query {:?} in timeline {:?}",
query_string, self.ztimelineid
);
let create = !(matches!(cmd, SafekeeperPostgresCommand::StartReplication { .. })
|| matches!(cmd, SafekeeperPostgresCommand::IdentifySystem));
let tenantid = self.ztenantid.context("tenantid is required")?;
let timelineid = self.ztimelineid.context("timelineid is required")?;
if self.timeline.is_none() {
self.timeline.set(
&self.conf,
ZTenantTimelineId::new(tenantid, timelineid),
create,
)?;
}
match cmd {
SafekeeperPostgresCommand::StartWalPush => ReceiveWalConn::new(pgb)
.run(self)
.context("failed to run ReceiveWalConn"),
SafekeeperPostgresCommand::StartReplication { start_lsn } => ReplicationConn::new(pgb)
.run(self, pgb, start_lsn)
.context("failed to run ReplicationConn"),
SafekeeperPostgresCommand::IdentifySystem => self.handle_identify_system(pgb),
SafekeeperPostgresCommand::JSONCtrl { ref cmd } => handle_json_ctrl(self, pgb, cmd),
}
.context(format!("timeline {timelineid}"))?;
Ok(())
}
}
impl SafekeeperPostgresHandler {
pub fn new(conf: SafeKeeperConf) -> Self {
SafekeeperPostgresHandler {
conf,
appname: None,
ztenantid: None,
ztimelineid: None,
timeline: None,
}
}
/// Shortcut for calling `process_msg` in the timeline.
pub fn process_safekeeper_msg(
&self,
msg: &ProposerAcceptorMessage,
) -> Result<Option<AcceptorProposerMessage>> {
self.timeline
.get()
.process_msg(msg)
.context("failed to process ProposerAcceptorMessage")
}
///
/// Handle IDENTIFY_SYSTEM replication command
///
fn handle_identify_system(&mut self, pgb: &mut PostgresBackend) -> Result<()> {
let lsn = if self.is_walproposer_recovery() {
// walproposer should get all local WAL until flush_lsn
self.timeline.get().get_end_of_wal()
} else {
// other clients shouldn't get any uncommitted WAL
self.timeline.get().get_state().0.commit_lsn
}
.to_string();
let sysid = self
.timeline
.get()
.get_state()
.1
.server
.system_id
.to_string();
let lsn_bytes = lsn.as_bytes();
let tli = PG_TLI.to_string();
let tli_bytes = tli.as_bytes();
let sysid_bytes = sysid.as_bytes();
pgb.write_message_noflush(&BeMessage::RowDescription(&[
RowDescriptor {
name: b"systemid",
typoid: TEXT_OID,
typlen: -1,
..Default::default()
},
RowDescriptor {
name: b"timeline",
typoid: INT4_OID,
typlen: 4,
..Default::default()
},
RowDescriptor {
name: b"xlogpos",
typoid: TEXT_OID,
typlen: -1,
..Default::default()
},
RowDescriptor {
name: b"dbname",
typoid: TEXT_OID,
typlen: -1,
..Default::default()
},
]))?
.write_message_noflush(&BeMessage::DataRow(&[
Some(sysid_bytes),
Some(tli_bytes),
Some(lsn_bytes),
None,
]))?
.write_message(&BeMessage::CommandComplete(b"IDENTIFY_SYSTEM"))?;
Ok(())
}
/// Returns true if current connection is a replication connection, originating
/// from a walproposer recovery function. This connection gets a special handling:
/// safekeeper must stream all local WAL till the flush_lsn, whether committed or not.
pub fn is_walproposer_recovery(&self) -> bool {
self.appname == Some("wal_proposer_recovery".to_string())
}
}