Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: Enable postgres continue ingestion #2290

Merged
merged 1 commit into from
Jan 3, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
94 changes: 62 additions & 32 deletions dozer-ingestion/postgres/src/connector.rs
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
use dozer_ingestion_connector::dozer_types::log::warn;
use dozer_ingestion_connector::{
async_trait,
dozer_types::{errors::internal::BoxedError, log::info, types::FieldType},
Expand All @@ -10,6 +11,7 @@ use rand::Rng;
use tokio_postgres::config::ReplicationMode;
use tokio_postgres::Config;

use crate::state::LsnWithSlot;
use crate::{
connection::validator::validate_connection,
iterator::PostgresIterator,
Expand Down Expand Up @@ -64,22 +66,44 @@ impl PostgresConnector {
}

fn get_lsn_with_offset_from_seq(
conn_name: String,
from_seq: Option<(u64, u64)>,
) -> Option<(PgLsn, u64)> {
from_seq.map_or_else(
|| {
info!("[{}] Starting replication", conn_name);
None
},
|(lsn, checkpoint)| {
info!(
"[{}] Starting replication from checkpoint ({}/{})",
conn_name, lsn, checkpoint
);
Some((PgLsn::from(lsn), checkpoint))
},
)
conn_name: &str,
tables: Vec<TableToIngest>,
) -> Option<LsnWithSlot> {
let m: Option<LsnWithSlot> = tables
.iter()
.filter_map(|table| {
if let Some(s) = &table.state {
match LsnWithSlot::try_from(s.clone()) {
Ok(state) => Some(state),
Err(e) => {
warn!(
"[{conn_name}] Failed to parse checkpoint: {error}",
conn_name = conn_name,
error = e
);
None
}
}
} else {
None
}
})
.collect::<Vec<LsnWithSlot>>()
.iter()
.max_by_key(|x| x.lsn)
.cloned();

if let Some(x) = &m {
info!(
"[{conn_name}] Last checkpoint {txid}({seq_in_tx}) in {slot_name}",
conn_name = conn_name,
txid = x.lsn.0,
seq_in_tx = x.lsn.1,
slot_name = x.slot_name
);
}

m
}
}

Expand Down Expand Up @@ -173,31 +197,37 @@ impl Connector for PostgresConnector {
ingestor: &Ingestor,
tables: Vec<TableToIngest>,
) -> Result<(), BoxedError> {
let client = helper::connect(self.replication_conn_config.clone()).await?;
let table_identifiers = tables
.iter()
.map(|table| TableIdentifier::new(table.schema.clone(), table.name.clone()))
.collect::<Vec<_>>();
self.create_publication(client, Some(&table_identifiers))
.await?;
let lsn_with_slot =
PostgresConnector::get_lsn_with_offset_from_seq(&self.name, tables.clone());
let slot_name = lsn_with_slot
.clone()
.map_or(self.get_slot_name(), |LsnWithSlot { slot_name, .. }| {
slot_name
});
let lsn = lsn_with_slot.map(|LsnWithSlot { lsn, .. }| lsn);

let lsn = PostgresConnector::get_lsn_with_offset_from_seq(self.name.clone(), None);
if lsn.is_none() {
let client = helper::connect(self.replication_conn_config.clone()).await?;
let table_identifiers = tables
.iter()
.map(|table| TableIdentifier::new(table.schema.clone(), table.name.clone()))
.collect::<Vec<_>>();
self.create_publication(client, Some(&table_identifiers))
.await?;
}

let tables = tables
.into_iter()
.map(|table| {
assert!(table.state.is_none());
ListOrFilterColumns {
schema: table.schema,
name: table.name,
columns: Some(table.column_names),
}
.map(|table| ListOrFilterColumns {
schema: table.schema,
name: table.name,
columns: Some(table.column_names),
})
.collect::<Vec<_>>();
let iterator = PostgresIterator::new(
self.name.clone(),
self.get_publication_name(),
self.get_slot_name(),
slot_name,
self.schema_helper.get_tables(Some(&tables)).await?,
self.replication_conn_config.clone(),
ingestor,
Expand Down
7 changes: 6 additions & 1 deletion dozer-ingestion/postgres/src/iterator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,12 @@ impl<'a> PostgresIteratorHandler<'a> {
// - When publication tables changes

// We clear inactive replication slots before starting replication
ReplicationSlotHelper::clear_inactive_slots(&mut client, REPLICATION_SLOT_PREFIX).await?;
ReplicationSlotHelper::clear_inactive_slots(
&mut client,
REPLICATION_SLOT_PREFIX,
Some(&details.slot_name),
)
.await?;

if self.lsn.is_none() {
debug!("\nCreating Slot....");
Expand Down
10 changes: 10 additions & 0 deletions dozer-ingestion/postgres/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ mod replication_slot_helper;
pub mod replicator;
mod schema;
pub mod snapshotter;
mod state;
#[cfg(test)]
pub mod test_utils;
#[cfg(test)]
Expand Down Expand Up @@ -142,6 +143,15 @@ pub enum PostgresConnectorError {
UnexpectedQueryMessageError,
}

#[derive(Error, Debug)]
pub enum PostgresStateError {
#[error("Failed to read lsn from state. Error: {0}")]
TrySliceError(#[from] std::array::TryFromSliceError),

#[error("Failed to convert slot name from state. Error: {0}")]
StringReadError(#[from] FromUtf8Error),
}

#[derive(Error, Debug)]
pub enum PostgresSchemaError {
#[error("Schema's '{0}' doesn't have primary key")]
Expand Down
8 changes: 7 additions & 1 deletion dozer-ingestion/postgres/src/replication_slot_helper.rs
Original file line number Diff line number Diff line change
Expand Up @@ -65,9 +65,15 @@ impl ReplicationSlotHelper {
pub async fn clear_inactive_slots(
client: &mut Client,
slot_name_prefix: &str,
current_slot_name: Option<&str>,
) -> Result<(), PostgresConnectorError> {
let condition = match current_slot_name {
Some(name) => format!("AND slot_name != '{name}'"),
None => "".to_string(),
};

let inactive_slots_query = format!(
r#"SELECT * FROM pg_replication_slots where active = false AND slot_name LIKE '{slot_name_prefix}%';"#
r#"SELECT * FROM pg_replication_slots where active = false AND slot_name LIKE '{slot_name_prefix}%' {condition};"#
);

let slots = client
Expand Down
15 changes: 6 additions & 9 deletions dozer-ingestion/postgres/src/replicator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@ use dozer_ingestion_connector::dozer_types::bytes;
use dozer_ingestion_connector::dozer_types::chrono::{TimeZone, Utc};
use dozer_ingestion_connector::dozer_types::log::{error, info};
use dozer_ingestion_connector::dozer_types::models::ingestion_types::IngestionMessage;
use dozer_ingestion_connector::dozer_types::node::RestartableState;
use dozer_ingestion_connector::futures::StreamExt;
use dozer_ingestion_connector::Ingestor;
use postgres_protocol::message::backend::ReplicationMessage::*;
Expand All @@ -16,6 +15,7 @@ use std::time::SystemTime;

use crate::connection::client::Client;
use crate::connection::helper::{self, is_network_failure};
use crate::state::encode_state;
use crate::xlog_mapper::XlogMapper;
use crate::PostgresConnectorError;

Expand Down Expand Up @@ -132,7 +132,11 @@ impl<'a> CDCHandler<'a> {
.handle_message(IngestionMessage::OperationEvent {
table_index,
op,
state: Some(encode_state(self.begin_lsn, self.seq_no)),
state: Some(encode_state(
self.begin_lsn,
self.seq_no,
self.slot_name.clone(),
)),
})
.await
.is_err()
Expand All @@ -156,13 +160,6 @@ impl<'a> CDCHandler<'a> {
}
}

fn encode_state(lsn: Lsn, seq_no: u64) -> RestartableState {
let mut state = vec![];
state.extend_from_slice(&lsn.to_be_bytes());
state.extend_from_slice(&seq_no.to_be_bytes());
state.into()
}

pub struct LogicalReplicationStream {
client: Client,
slot_name: String,
Expand Down
35 changes: 35 additions & 0 deletions dozer-ingestion/postgres/src/state.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
use crate::connector::REPLICATION_SLOT_PREFIX;
use crate::PostgresStateError;
use dozer_ingestion_connector::dozer_types::node::RestartableState;
use postgres_protocol::Lsn;
use postgres_types::PgLsn;

#[derive(Clone)]
pub struct LsnWithSlot {
pub lsn: (PgLsn, u64),
pub slot_name: String,
}

pub fn encode_state(lsn: Lsn, seq_no: u64, slot_name: String) -> RestartableState {
let slot_name_suffix = slot_name.replace(REPLICATION_SLOT_PREFIX, "");
let mut state = vec![];
state.extend_from_slice(&lsn.to_be_bytes());
state.extend_from_slice(&seq_no.to_be_bytes());
state.extend_from_slice(slot_name_suffix.as_bytes());
state.into()
}

impl TryFrom<RestartableState> for LsnWithSlot {
type Error = PostgresStateError;

fn try_from(state: RestartableState) -> Result<Self, Self::Error> {
let lsn = Lsn::from_be_bytes(state.0[0..8].try_into()?);
let seq_no = u64::from_be_bytes(state.0[8..16].try_into()?);
let slot_name = String::from_utf8(state.0[16..].into())?;

Ok(LsnWithSlot {
lsn: (PgLsn::from(lsn), seq_no),
slot_name: format!("{REPLICATION_SLOT_PREFIX}{slot_name}"),
})
}
}
Loading