mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-06 04:52:55 +00:00
Consolidate PG proto parsing-deparsing and backend code.
Now postgres_backend communicates with the client, passing queries to the provided handler; we have two currently, for wal_acceptor and pageserver. Now BytesMut is again used for writing data to avoid manual message length calculation. ref #118
This commit is contained in:
@@ -1,20 +1,18 @@
|
||||
//! This module implements the replication protocol, starting with the
|
||||
//! "START REPLICATION" message.
|
||||
//! This module implements the streaming side of replication protocol, starting
|
||||
//! with the "START REPLICATION" message.
|
||||
|
||||
use crate::pq_protocol::{BeMessage, FeMessage};
|
||||
use crate::send_wal::SendWalConn;
|
||||
use crate::send_wal::SendWalHandler;
|
||||
use crate::timeline::{Timeline, TimelineTools};
|
||||
use crate::WalAcceptorConf;
|
||||
use anyhow::{anyhow, Result};
|
||||
use bytes::{BufMut, Bytes, BytesMut};
|
||||
use bytes::Bytes;
|
||||
use log::*;
|
||||
use postgres_ffi::xlog_utils::{get_current_timestamp, TimestampTz, XLogFileName, MAX_SEND_SIZE};
|
||||
use regex::Regex;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use std::cmp::min;
|
||||
use std::fs::File;
|
||||
use std::io::{BufReader, Read, Seek, SeekFrom, Write};
|
||||
use std::net::{Shutdown, TcpStream};
|
||||
use std::io::{BufReader, Read, Seek, SeekFrom};
|
||||
use std::net::TcpStream;
|
||||
use std::path::Path;
|
||||
use std::sync::Arc;
|
||||
use std::thread::sleep;
|
||||
@@ -22,10 +20,9 @@ use std::time::Duration;
|
||||
use std::{str, thread};
|
||||
use zenith_utils::bin_ser::BeSer;
|
||||
use zenith_utils::lsn::Lsn;
|
||||
use zenith_utils::postgres_backend::PostgresBackend;
|
||||
use zenith_utils::pq_proto::{BeMessage, FeMessage, XLogDataBody};
|
||||
|
||||
const XLOG_HDR_SIZE: usize = 1 + 8 * 3; /* 'w' + startPos + walEnd + timestamp */
|
||||
const LIBPQ_HDR_SIZE: usize = 5; /* 1 byte with message type + 4 bytes length */
|
||||
const LIBPQ_MSG_SIZE_OFFS: usize = 1;
|
||||
pub const END_REPLICATION_MARKER: Lsn = Lsn::MAX;
|
||||
|
||||
type FullTransactionId = u64;
|
||||
@@ -40,37 +37,16 @@ pub struct HotStandbyFeedback {
|
||||
|
||||
/// A network connection that's speaking the replication protocol.
|
||||
pub struct ReplicationConn {
|
||||
timeline: Option<Arc<Timeline>>,
|
||||
/// Postgres connection, buffered input
|
||||
///
|
||||
/// This is an `Option` because we will spawn a background thread that will
|
||||
/// `take` it from us.
|
||||
stream_in: Option<BufReader<TcpStream>>,
|
||||
/// Postgres connection, output
|
||||
stream_out: TcpStream,
|
||||
/// wal acceptor configuration
|
||||
conf: WalAcceptorConf,
|
||||
/// assigned application name
|
||||
appname: Option<String>,
|
||||
}
|
||||
|
||||
// Separate thread is reading keepalives from the socket. When main one
|
||||
// finishes, tell it to get down by shutdowning the socket.
|
||||
impl Drop for ReplicationConn {
|
||||
fn drop(&mut self) {
|
||||
let _res = self.stream_out.shutdown(Shutdown::Both);
|
||||
}
|
||||
}
|
||||
|
||||
impl ReplicationConn {
|
||||
/// Create a new `SendWal`, consuming the `Connection`.
|
||||
pub fn new(conn: SendWalConn) -> Self {
|
||||
/// Create a new `ReplicationConn`
|
||||
pub fn new(pgb: &mut PostgresBackend) -> Self {
|
||||
Self {
|
||||
timeline: conn.timeline,
|
||||
stream_in: Some(conn.stream_in),
|
||||
stream_out: conn.stream_out,
|
||||
conf: conn.conf,
|
||||
appname: None,
|
||||
stream_in: pgb.take_stream_in(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -82,9 +58,9 @@ impl ReplicationConn {
|
||||
// Wait for replica's feedback.
|
||||
// We only handle `CopyData` messages. Anything else is ignored.
|
||||
loop {
|
||||
match FeMessage::read_from(&mut stream_in)? {
|
||||
FeMessage::CopyData(m) => {
|
||||
let feedback = HotStandbyFeedback::des(&m.body)?;
|
||||
match FeMessage::read(&mut stream_in)? {
|
||||
Some(FeMessage::CopyData(m)) => {
|
||||
let feedback = HotStandbyFeedback::des(&m)?;
|
||||
timeline.add_hs_feedback(feedback)
|
||||
}
|
||||
msg => {
|
||||
@@ -128,9 +104,14 @@ impl ReplicationConn {
|
||||
///
|
||||
/// Handle START_REPLICATION replication command
|
||||
///
|
||||
pub fn run(&mut self, cmd: &Bytes) -> Result<()> {
|
||||
pub fn run(
|
||||
&mut self,
|
||||
swh: &mut SendWalHandler,
|
||||
pgb: &mut PostgresBackend,
|
||||
cmd: &Bytes,
|
||||
) -> Result<()> {
|
||||
// spawn the background thread which receives HotStandbyFeedback messages.
|
||||
let bg_timeline = Arc::clone(self.timeline.get());
|
||||
let bg_timeline = Arc::clone(swh.timeline.get());
|
||||
let bg_stream_in = self.stream_in.take().unwrap();
|
||||
|
||||
thread::spawn(move || {
|
||||
@@ -143,7 +124,7 @@ impl ReplicationConn {
|
||||
|
||||
let mut wal_seg_size: usize;
|
||||
loop {
|
||||
wal_seg_size = self.timeline.get().get_info().server.wal_seg_size as usize;
|
||||
wal_seg_size = swh.timeline.get().get_info().server.wal_seg_size as usize;
|
||||
if wal_seg_size == 0 {
|
||||
error!("Can not start replication before connecting to wal_proposer");
|
||||
sleep(Duration::from_secs(1));
|
||||
@@ -151,19 +132,17 @@ impl ReplicationConn {
|
||||
break;
|
||||
}
|
||||
}
|
||||
let (wal_end, timeline) = self.timeline.find_end_of_wal(&self.conf.data_dir, false);
|
||||
let (wal_end, timeline) = swh.timeline.find_end_of_wal(&swh.conf.data_dir, false);
|
||||
if start_pos == Lsn(0) {
|
||||
start_pos = wal_end;
|
||||
}
|
||||
if stop_pos == Lsn(0) && self.appname == Some("wal_proposer_recovery".to_string()) {
|
||||
if stop_pos == Lsn(0) && swh.appname == Some("wal_proposer_recovery".to_string()) {
|
||||
stop_pos = wal_end;
|
||||
}
|
||||
info!("Start replication from {} till {}", start_pos, stop_pos);
|
||||
|
||||
let mut outbuf = BytesMut::new();
|
||||
BeMessage::write(&mut outbuf, &BeMessage::Copy);
|
||||
self.send(&outbuf)?;
|
||||
outbuf.clear();
|
||||
// switch to copy
|
||||
pgb.write_message(&BeMessage::CopyBothResponse)?;
|
||||
|
||||
let mut end_pos: Lsn;
|
||||
let mut wal_file: Option<File> = None;
|
||||
@@ -179,7 +158,7 @@ impl ReplicationConn {
|
||||
end_pos = stop_pos;
|
||||
} else {
|
||||
/* normal mode */
|
||||
let timeline = self.timeline.get();
|
||||
let timeline = swh.timeline.get();
|
||||
end_pos = timeline.wait_for_lsn(start_pos);
|
||||
}
|
||||
if end_pos == END_REPLICATION_MARKER {
|
||||
@@ -193,8 +172,8 @@ impl ReplicationConn {
|
||||
// Open a new file.
|
||||
let segno = start_pos.segment_number(wal_seg_size);
|
||||
let wal_file_name = XLogFileName(timeline, segno, wal_seg_size);
|
||||
let timeline_id = self.timeline.get().timelineid.to_string();
|
||||
let wal_file_path = self.conf.data_dir.join(timeline_id).join(wal_file_name);
|
||||
let timeline_id = swh.timeline.get().timelineid.to_string();
|
||||
let wal_file_path = swh.conf.data_dir.join(timeline_id).join(wal_file_name);
|
||||
Self::open_wal_file(&wal_file_path)?
|
||||
}
|
||||
};
|
||||
@@ -207,32 +186,19 @@ impl ReplicationConn {
|
||||
let send_size = min(send_size, wal_seg_size - xlogoff);
|
||||
let send_size = min(send_size, MAX_SEND_SIZE);
|
||||
|
||||
let msg_size = LIBPQ_HDR_SIZE + XLOG_HDR_SIZE + send_size;
|
||||
|
||||
// Read some data from the file.
|
||||
let mut file_buf = vec![0u8; send_size];
|
||||
file.seek(SeekFrom::Start(xlogoff as u64))?;
|
||||
file.read_exact(&mut file_buf)?;
|
||||
|
||||
// Write some data to the network socket.
|
||||
// FIXME: turn these into structs.
|
||||
// 'd' is CopyData;
|
||||
// 'w' is "WAL records"
|
||||
// https://www.postgresql.org/docs/9.1/protocol-message-formats.html
|
||||
// src/backend/replication/walreceiver.c
|
||||
outbuf.clear();
|
||||
outbuf.put_u8(b'd');
|
||||
outbuf.put_u32((msg_size - LIBPQ_MSG_SIZE_OFFS) as u32);
|
||||
outbuf.put_u8(b'w');
|
||||
outbuf.put_u64(start_pos.0);
|
||||
outbuf.put_u64(end_pos.0);
|
||||
outbuf.put_u64(get_current_timestamp());
|
||||
pgb.write_message(&BeMessage::XLogData(XLogDataBody {
|
||||
wal_start: start_pos.0,
|
||||
wal_end: end_pos.0,
|
||||
timestamp: get_current_timestamp(),
|
||||
data: &file_buf,
|
||||
}))?;
|
||||
|
||||
assert!(outbuf.len() + file_buf.len() == msg_size);
|
||||
// This thread has exclusive access to the TcpStream, so it's fine
|
||||
// to do this as two separate calls.
|
||||
self.send(&outbuf)?;
|
||||
self.send(&file_buf)?;
|
||||
start_pos += send_size as u64;
|
||||
|
||||
debug!("Sent WAL to page server up to {}", end_pos);
|
||||
@@ -245,10 +211,4 @@ impl ReplicationConn {
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Send messages on the network.
|
||||
fn send(&mut self, buf: &[u8]) -> Result<()> {
|
||||
self.stream_out.write_all(buf.as_ref())?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,111 +1,69 @@
|
||||
//! This implements the libpq replication protocol between wal_acceptor
|
||||
//! and replicas/pagers
|
||||
//! Part of WAL acceptor pretending to be Postgres, streaming xlog to
|
||||
//! pageserver/any other consumer.
|
||||
//!
|
||||
|
||||
use crate::pq_protocol::{
|
||||
BeMessage, FeMessage, FeStartupMessage, RowDescriptor, StartupRequestCode,
|
||||
};
|
||||
use crate::replication::ReplicationConn;
|
||||
use crate::timeline::{Timeline, TimelineTools};
|
||||
use crate::WalAcceptorConf;
|
||||
use anyhow::{bail, Result};
|
||||
use bytes::BytesMut;
|
||||
use log::*;
|
||||
use std::io::{BufReader, Write};
|
||||
use std::net::{SocketAddr, TcpStream};
|
||||
use bytes::Bytes;
|
||||
use pageserver::ZTimelineId;
|
||||
use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
use zenith_utils::postgres_backend;
|
||||
use zenith_utils::postgres_backend::PostgresBackend;
|
||||
use zenith_utils::pq_proto::{BeMessage, FeStartupMessage, RowDescriptor};
|
||||
|
||||
/// A network connection that's speaking the libpq replication protocol.
|
||||
pub struct SendWalConn {
|
||||
pub timeline: Option<Arc<Timeline>>,
|
||||
/// Postgres connection, buffered input
|
||||
pub stream_in: BufReader<TcpStream>,
|
||||
/// Postgres connection, output
|
||||
pub stream_out: TcpStream,
|
||||
/// The cached result of socket.peer_addr()
|
||||
pub peer_addr: SocketAddr,
|
||||
/// Handler for streaming WAL from acceptor
|
||||
pub struct SendWalHandler {
|
||||
/// wal acceptor configuration
|
||||
pub conf: WalAcceptorConf,
|
||||
/// assigned application name
|
||||
appname: Option<String>,
|
||||
pub appname: Option<String>,
|
||||
pub timeline: Option<Arc<Timeline>>,
|
||||
}
|
||||
|
||||
impl SendWalConn {
|
||||
/// Create a new `SendWal`, consuming the `Connection`.
|
||||
pub fn new(socket: TcpStream, conf: WalAcceptorConf) -> Result<Self> {
|
||||
let peer_addr = socket.peer_addr()?;
|
||||
let conn = SendWalConn {
|
||||
timeline: None,
|
||||
stream_in: BufReader::new(socket.try_clone()?),
|
||||
stream_out: socket,
|
||||
peer_addr,
|
||||
conf,
|
||||
appname: None,
|
||||
};
|
||||
Ok(conn)
|
||||
impl postgres_backend::Handler for SendWalHandler {
|
||||
fn startup(&mut self, _pgb: &mut PostgresBackend, sm: &FeStartupMessage) -> Result<()> {
|
||||
match sm.params.get("ztimelineid") {
|
||||
Some(ref ztimelineid) => {
|
||||
let ztlid = ZTimelineId::from_str(ztimelineid)?;
|
||||
self.timeline.set(ztlid)?;
|
||||
}
|
||||
_ => bail!("timelineid is required"),
|
||||
}
|
||||
if let Some(app_name) = sm.params.get("application_name") {
|
||||
self.appname = Some(app_name.clone());
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
///
|
||||
/// Send WAL to replica or WAL receiver using standard libpq replication protocol
|
||||
///
|
||||
pub fn run(mut self) -> Result<()> {
|
||||
let peer_addr = self.peer_addr;
|
||||
info!("WAL sender to {:?} is started", peer_addr);
|
||||
|
||||
// Handle the startup message first.
|
||||
|
||||
let m = FeStartupMessage::read_from(&mut self.stream_in)?;
|
||||
trace!("got startup message {:?}", m);
|
||||
match m.kind {
|
||||
StartupRequestCode::NegotiateGss | StartupRequestCode::NegotiateSsl => {
|
||||
let mut buf = BytesMut::new();
|
||||
BeMessage::write(&mut buf, &BeMessage::Negotiate);
|
||||
info!("SSL requested");
|
||||
self.stream_out.write_all(&buf)?;
|
||||
}
|
||||
StartupRequestCode::Normal => {
|
||||
let mut buf = BytesMut::new();
|
||||
BeMessage::write(&mut buf, &BeMessage::AuthenticationOk);
|
||||
BeMessage::write(&mut buf, &BeMessage::ReadyForQuery);
|
||||
self.stream_out.write_all(&buf)?;
|
||||
self.timeline.set(m.timelineid)?;
|
||||
self.appname = m.appname;
|
||||
}
|
||||
StartupRequestCode::Cancel => return Ok(()),
|
||||
fn process_query(&mut self, pgb: &mut PostgresBackend, query_string: Bytes) -> Result<()> {
|
||||
if query_string.starts_with(b"IDENTIFY_SYSTEM") {
|
||||
self.handle_identify_system(pgb)?;
|
||||
Ok(())
|
||||
} else if query_string.starts_with(b"START_REPLICATION") {
|
||||
ReplicationConn::new(pgb).run(self, pgb, &query_string)?;
|
||||
Ok(())
|
||||
} else {
|
||||
bail!("Unexpected command {:?}", query_string);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
loop {
|
||||
let msg = FeMessage::read_from(&mut self.stream_in)?;
|
||||
match msg {
|
||||
FeMessage::Query(q) => {
|
||||
trace!("got query {:?}", q.body);
|
||||
|
||||
if q.body.starts_with(b"IDENTIFY_SYSTEM") {
|
||||
self.handle_identify_system()?;
|
||||
} else if q.body.starts_with(b"START_REPLICATION") {
|
||||
// Create a new replication object, consuming `self`.
|
||||
ReplicationConn::new(self).run(&q.body)?;
|
||||
break;
|
||||
} else {
|
||||
bail!("Unexpected command {:?}", q.body);
|
||||
}
|
||||
}
|
||||
FeMessage::Terminate => {
|
||||
break;
|
||||
}
|
||||
_ => {
|
||||
bail!("unexpected message");
|
||||
}
|
||||
}
|
||||
impl SendWalHandler {
|
||||
pub fn new(conf: WalAcceptorConf) -> Self {
|
||||
SendWalHandler {
|
||||
conf,
|
||||
appname: None,
|
||||
timeline: None,
|
||||
}
|
||||
info!("WAL sender to {:?} is finished", peer_addr);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
///
|
||||
/// Handle IDENTIFY_SYSTEM replication command
|
||||
///
|
||||
fn handle_identify_system(&mut self) -> Result<()> {
|
||||
fn handle_identify_system(&mut self, pgb: &mut PostgresBackend) -> Result<()> {
|
||||
let (start_pos, timeline) = self.timeline.find_end_of_wal(&self.conf.data_dir, false);
|
||||
let lsn = start_pos.to_string();
|
||||
let tli = timeline.to_string();
|
||||
@@ -114,42 +72,39 @@ impl SendWalConn {
|
||||
let tli_bytes = tli.as_bytes();
|
||||
let sysid_bytes = sysid.as_bytes();
|
||||
|
||||
let mut outbuf = BytesMut::new();
|
||||
BeMessage::write(
|
||||
&mut outbuf,
|
||||
&BeMessage::RowDescription(&[
|
||||
RowDescriptor {
|
||||
name: b"systemid\0",
|
||||
typoid: 25,
|
||||
typlen: -1,
|
||||
},
|
||||
RowDescriptor {
|
||||
name: b"timeline\0",
|
||||
typoid: 23,
|
||||
typlen: 4,
|
||||
},
|
||||
RowDescriptor {
|
||||
name: b"xlogpos\0",
|
||||
typoid: 25,
|
||||
typlen: -1,
|
||||
},
|
||||
RowDescriptor {
|
||||
name: b"dbname\0",
|
||||
typoid: 25,
|
||||
typlen: -1,
|
||||
},
|
||||
]),
|
||||
);
|
||||
BeMessage::write(
|
||||
&mut outbuf,
|
||||
&BeMessage::DataRow(&[Some(sysid_bytes), Some(tli_bytes), Some(lsn_bytes), None]),
|
||||
);
|
||||
BeMessage::write(
|
||||
&mut outbuf,
|
||||
&BeMessage::CommandComplete(b"IDENTIFY_SYSTEM\0"),
|
||||
);
|
||||
BeMessage::write(&mut outbuf, &BeMessage::ReadyForQuery);
|
||||
self.stream_out.write_all(&outbuf)?;
|
||||
pgb.write_message_noflush(&BeMessage::RowDescription(&[
|
||||
RowDescriptor {
|
||||
name: b"systemid",
|
||||
typoid: 25,
|
||||
typlen: -1,
|
||||
..Default::default()
|
||||
},
|
||||
RowDescriptor {
|
||||
name: b"timeline",
|
||||
typoid: 23,
|
||||
typlen: 4,
|
||||
..Default::default()
|
||||
},
|
||||
RowDescriptor {
|
||||
name: b"xlogpos",
|
||||
typoid: 25,
|
||||
typlen: -1,
|
||||
..Default::default()
|
||||
},
|
||||
RowDescriptor {
|
||||
name: b"dbname",
|
||||
typoid: 25,
|
||||
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(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -9,8 +9,9 @@ use std::net::{TcpListener, TcpStream};
|
||||
use std::thread;
|
||||
|
||||
use crate::receive_wal::ReceiveWalConn;
|
||||
use crate::send_wal::SendWalConn;
|
||||
use crate::send_wal::SendWalHandler;
|
||||
use crate::WalAcceptorConf;
|
||||
use zenith_utils::postgres_backend::PostgresBackend;
|
||||
|
||||
/// Accept incoming TCP connections and spawn them into a background thread.
|
||||
pub fn thread_main(conf: WalAcceptorConf) -> Result<()> {
|
||||
@@ -48,7 +49,10 @@ fn handle_socket(mut socket: TcpStream, conf: WalAcceptorConf) -> Result<()> {
|
||||
socket.read_exact(&mut [0u8; 4])?;
|
||||
ReceiveWalConn::new(socket, conf)?.run()?; // internal protocol between wal_proposer and wal_acceptor
|
||||
} else {
|
||||
SendWalConn::new(socket, conf)?.run()?; // libpq replication protocol between wal_acceptor and replicas/pagers
|
||||
let mut conn_handler = SendWalHandler::new(conf);
|
||||
let mut pgbackend = PostgresBackend::new(socket)?;
|
||||
// libpq replication protocol between wal_acceptor and replicas/pagers
|
||||
pgbackend.run(&mut conn_handler)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user