From 5380e8a4b10e87bb69806f5ea35181ad7fcbce93 Mon Sep 17 00:00:00 2001 From: Hou Zhijie Date: Fri, 17 Jan 2025 14:08:02 +0800 Subject: [PATCH 1/7] Maintain the oldest non removeable tranasction ID by apply worker This set of patches aims to support the detection of update_deleted conflicts, which occur when the apply worker cannot find the target tuple to be updated (e.g., the tuple has been removed by a different origin). To detect this conflict consistently and correctly, we must ensure that tuples deleted by other origins are not prematurely removed by VACUUM before conflict detection. If these tuples are removed too soon, a different conflict might arise and be resolved incorrectly, causing data inconsistency between nodes. To achieve this, we will retain the dead tuples on the subscriber for some period. The concept is that dead tuples are useful for detecting conflicts only during the application of concurrent transactions from remote nodes. After applying and flushing all remote transactions that occurred concurrently with the tuple DELETE, any subsequent UPDATE from a remote node should have a later timestamp. In such cases, it is acceptable to detect an update_missing scenario and convert the UPDATE to an INSERT when applying it. But, for concurrent remote transactions with earlier timestamps than the DELETE, detecting update_deleted is necessary, as the UPDATEs in remote transactions should be ignored if their timestamp is earlier than that of the dead tuples. We assume that the appropriate resolution for update_deleted conflicts, to achieve eventual consistency, is the last-update-win strategy. This means that when detecting the update_deleted conflict, and the remote update has a later timestamp, the resolution would be to convert the UPDATE to an INSERT. Remote updates with earlier timestamps compared to the dead tuples will be disregarded. To implement this, an additional replication slot named pg_conflict_detection will be created on the subscriber side and maintained by the launcher. This slot will be used to retain dead tuples. Each apply worker will maintain its own non-removable transaction ID, while the launcher collects these IDs to determine whether to advance the xmin value of the replication slot. The process of advancing the non-removable transaction ID in the apply worker involves: 1) Call GetOldestActiveTransactionId() to take oldestRunningXid as the candidate xid. 2) Send a message to the walsender requesting the publisher status, which includes the latest WAL write position and information about transactions that are in the commit phase. 3) Wait for the status from the walsender. After receiving the first status, do not proceed if there are concurrent remote transactions that are still in the commit phase. These transactions might have been assigned an earlier commit timestamp but have not yet written the commit WAL record. Continue to request the publisher status until all these transactions have completed. 4) Advance the non-removable transaction ID if the current flush location has reached or surpassed the last received WAL position. These steps are repeated at intervals that are dynamically adjusted based on whether a new transaction ID has been assigned since the last advancement. This mechanism ensures that dead tuples are not removed until all concurrent transactions have been applied. It works for both bidirectional and non-bidirectional replication scenarios. Since the mechanism relies on a single replication slot, it not only assists in retaining dead tuples but also preserves commit timestamps and origin data. These information will be displayed in the additional logs generated for logical replication conflicts. Furthermore, the preserved commit timestamps and origin data are essential for consistently detecting update_origin_differs conflicts. This patch allows each apply worker to maintain the non-removable transaction ID in the shared memory following the steps described above. The actual replication slot management is implemented in the following patches. --- doc/src/sgml/protocol.sgml | 90 ++++ src/backend/access/transam/twophase.c | 26 +- src/backend/access/transam/xact.c | 6 +- src/backend/replication/logical/launcher.c | 1 + src/backend/replication/logical/worker.c | 491 ++++++++++++++++++++- src/backend/replication/walsender.c | 54 +++ src/backend/storage/ipc/procarray.c | 59 +++ src/include/replication/worker_internal.h | 17 + src/include/storage/proc.h | 8 + src/include/storage/procarray.h | 1 + src/tools/pgindent/typedefs.list | 2 + 11 files changed, 745 insertions(+), 10 deletions(-) diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml index 65bbfc34f50b..aab1fc06de23 100644 --- a/doc/src/sgml/protocol.sgml +++ b/doc/src/sgml/protocol.sgml @@ -2638,6 +2638,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;" + + + Primary status update (B) + + + + Byte1('s') + + + Identifies the message as a primary status update. + + + + + + Int64 + + + The latest WAL write position on the server. + + + + + + Int32 + + + The oldest transaction ID that is currently in the commit + phase on the server. + + + + + + Int32 + + + The next transaction ID to be assigned on the server. + + + + + + Int32 + + + The epoch of the next transaction ID to be assigned. + + + + + + Int64 + + + The server's system clock at the time of transmission, as + microseconds since midnight on 2000-01-01. + + + + + + @@ -2782,6 +2845,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;" + + + Request primary status update (F) + + + + Byte1('p') + + + Identifies the message as a request for a primary status update. + + + + + + Int64 + + + The client's system clock at the time of transmission, as + microseconds since midnight on 2000-01-01. + + + + + + + diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c index 73a80559194e..41fb4fc5025c 100644 --- a/src/backend/access/transam/twophase.c +++ b/src/backend/access/transam/twophase.c @@ -1181,7 +1181,11 @@ EndPrepare(GlobalTransaction gxact) * starting immediately after the WAL record is inserted could complete * without fsync'ing our state file. (This is essentially the same kind * of race condition as the COMMIT-to-clog-write case that - * RecordTransactionCommit uses DELAY_CHKPT_START for; see notes there.) + * RecordTransactionCommit uses DELAY_CHKPT_IN_COMMIT for; see notes + * there.). Note that DELAY_CHKPT_IN_COMMIT is used to find transactions + * in the critical commit section. We need to know about such transactions + * for conflict detection and resolution in logical replication. See + * GetOldestTransactionIdInCommit and its use. * * We save the PREPARE record's location in the gxact for later use by * CheckPointTwoPhase. @@ -2286,7 +2290,7 @@ ProcessTwoPhaseBuffer(TransactionId xid, * RecordTransactionCommitPrepared * * This is basically the same as RecordTransactionCommit (q.v. if you change - * this function): in particular, we must set DELAY_CHKPT_START to avoid a + * this function): in particular, we must set DELAY_CHKPT_IN_COMMIT to avoid a * race condition. * * We know the transaction made at least one XLOG entry (its PREPARE), @@ -2306,7 +2310,7 @@ RecordTransactionCommitPrepared(TransactionId xid, const char *gid) { XLogRecPtr recptr; - TimestampTz committs = GetCurrentTimestamp(); + TimestampTz committs; bool replorigin; /* @@ -2319,8 +2323,18 @@ RecordTransactionCommitPrepared(TransactionId xid, START_CRIT_SECTION(); /* See notes in RecordTransactionCommit */ - Assert((MyProc->delayChkptFlags & DELAY_CHKPT_START) == 0); - MyProc->delayChkptFlags |= DELAY_CHKPT_START; + Assert((MyProc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0); + MyProc->delayChkptFlags |= DELAY_CHKPT_IN_COMMIT; + + /* + * Note it is important to set committs value after marking ourselves as + * in the commit critical section (DELAY_CHKPT_IN_COMMIT). This is because + * we want to ensure all such transactions are finished before we allow + * the logical replication client to advance its xid which is used to hold + * back dead rows for conflict detection. See + * maybe_advance_nonremovable_xid. + */ + committs = GetCurrentTimestamp(); /* * Emit the XLOG commit record. Note that we mark 2PC commits as @@ -2369,7 +2383,7 @@ RecordTransactionCommitPrepared(TransactionId xid, TransactionIdCommitTree(xid, nchildren, children); /* Checkpoint can proceed now */ - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION(); diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c index b885513f7654..dfc5108da3c2 100644 --- a/src/backend/access/transam/xact.c +++ b/src/backend/access/transam/xact.c @@ -1432,9 +1432,9 @@ RecordTransactionCommit(void) * modifying it. This makes checkpoint's determination of which xacts * are delaying the checkpoint a bit fuzzy, but it doesn't matter. */ - Assert((MyProc->delayChkptFlags & DELAY_CHKPT_START) == 0); + Assert((MyProc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0); START_CRIT_SECTION(); - MyProc->delayChkptFlags |= DELAY_CHKPT_START; + MyProc->delayChkptFlags |= DELAY_CHKPT_IN_COMMIT; /* * Insert the commit XLOG record. @@ -1537,7 +1537,7 @@ RecordTransactionCommit(void) */ if (markXidCommitted) { - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION(); } diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c index 10677da56b2b..8e42787a4265 100644 --- a/src/backend/replication/logical/launcher.c +++ b/src/backend/replication/logical/launcher.c @@ -441,6 +441,7 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype, worker->stream_fileset = NULL; worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid; worker->parallel_apply = is_parallel_apply_worker; + worker->oldest_nonremovable_xid = InvalidFullTransactionId; worker->last_lsn = InvalidXLogRecPtr; TIMESTAMP_NOBEGIN(worker->last_send_time); TIMESTAMP_NOBEGIN(worker->last_recv_time); diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c index 4151a4b2a96b..ffbd4e3a02c7 100644 --- a/src/backend/replication/logical/worker.c +++ b/src/backend/replication/logical/worker.c @@ -179,6 +179,7 @@ #include "storage/buffile.h" #include "storage/ipc.h" #include "storage/lmgr.h" +#include "storage/procarray.h" #include "tcop/tcopprot.h" #include "utils/acl.h" #include "utils/dynahash.h" @@ -275,6 +276,63 @@ typedef enum TRANS_PARALLEL_APPLY, } TransApplyAction; +/* + * The phases involved in advancing the non-removable transaction ID. + * + * See maybe_advance_nonremovable_xid() for details of the transition + * between these phases. + */ +typedef enum +{ + RCI_GET_CANDIDATE_XID, + RCI_REQUEST_PUBLISHER_STATUS, + RCI_WAIT_FOR_PUBLISHER_STATUS, + RCI_WAIT_FOR_LOCAL_FLUSH +} RetainConflictInfoPhase; + +/* + * Critical information for managing phase transitions within the + * RetainConflictInfoPhase. + */ +typedef struct RetainConflictInfoData +{ + RetainConflictInfoPhase phase; /* current phase */ + XLogRecPtr remote_lsn; /* WAL write position on the publisher */ + TransactionId remote_oldestxid; /* oldest transaction ID that was in the + * commit phase on the publisher */ + TransactionId remote_nextxid; /* next transaction ID to be assigned on + * the publisher */ + uint32 remote_epoch; /* epoch of remote_nextxid */ + FullTransactionId last_phase_at; /* publisher transaction ID that must + * be awaited to complete before + * entering the final phase + * (RCI_WAIT_FOR_LOCAL_FLUSH) */ + FullTransactionId candidate_xid; /* candidate for the non-removable + * transaction ID */ + TimestampTz reply_time; /* when the publisher responds with status */ + TimestampTz flushpos_update_time; /* when the remote flush position was + * updated in final phase + * (RCI_WAIT_FOR_LOCAL_FLUSH) */ + + /* + * The following fields are used to determine the timing for the next + * round of transaction ID advancement. + */ + TimestampTz last_recv_time; /* when the last message was received */ + TimestampTz candidate_xid_time; /* when the candidate_xid is decided */ + int xid_advance_interval; /* how much time (ms) to wait before + * attempting to advance the + * non-removable transaction ID */ +} RetainConflictInfoData; + +/* + * The minimum (100ms) and maximum (3 minutes) intervals for advancing + * non-removable transaction IDs. The maximum interval is a bit arbitrary but + * is sufficient to not cause any undue network traffic. + */ +#define MIN_XID_ADVANCE_INTERVAL 100 +#define MAX_XID_ADVANCE_INTERVAL 180000 + /* errcontext tracker */ static ApplyErrorCallbackArg apply_error_callback_arg = { @@ -339,6 +397,13 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr; /* BufFile handle of the current streaming file */ static BufFile *stream_fd = NULL; +/* + * The remote WAL position that has been applied and flushed locally. We + * record this information while sending feedback to the server and use this + * both while sending feedback and advancing oldest_nonremovable_xid. + */ +static XLogRecPtr last_flushpos = InvalidXLogRecPtr; + typedef struct SubXactInfo { TransactionId xid; /* XID of the subxact */ @@ -379,6 +444,16 @@ static void stream_close_file(void); static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply); +static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data, + bool status_received); +static void get_candidate_xid(RetainConflictInfoData *data); +static void request_publisher_status(RetainConflictInfoData *data); +static void wait_for_publisher_status(RetainConflictInfoData *data, + bool status_received); +static void wait_for_local_flush(RetainConflictInfoData *data); +static void adjust_xid_advance_interval(RetainConflictInfoData *data, + bool new_xid_found); + static void apply_handle_commit_internal(LogicalRepCommitData *commit_data); static void apply_handle_insert_internal(ApplyExecutionData *edata, ResultRelInfo *relinfo, @@ -3584,6 +3659,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received) bool ping_sent = false; TimeLineID tli; ErrorContextCallback errcallback; + RetainConflictInfoData data = {0}; /* * Init the ApplyMessageContext which we clean up after each replication @@ -3662,6 +3738,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received) last_recv_timestamp = GetCurrentTimestamp(); ping_sent = false; + data.last_recv_time = last_recv_timestamp; + /* Ensure we are reading the data into our memory context. */ MemoryContextSwitchTo(ApplyMessageContext); @@ -3688,6 +3766,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received) UpdateWorkerStats(last_received, send_time, false); apply_dispatch(&s); + + maybe_advance_nonremovable_xid(&data, false); } else if (c == 'k') { @@ -3703,8 +3783,32 @@ LogicalRepApplyLoop(XLogRecPtr last_received) last_received = end_lsn; send_feedback(last_received, reply_requested, false); + + maybe_advance_nonremovable_xid(&data, false); + UpdateWorkerStats(last_received, timestamp, true); } + else if (c == 's') /* Primary status update */ + { + data.remote_lsn = pq_getmsgint64(&s); + data.remote_oldestxid = pq_getmsgint(&s, 4); + data.remote_nextxid = pq_getmsgint(&s, 4); + data.remote_epoch = pq_getmsgint(&s, 4); + data.reply_time = pq_getmsgint64(&s); + + /* + * This should never happen, see + * ProcessStandbyPSRequestMessage. But if it happens + * due to a bug, we don't want to proceed as it can + * incorrectly advance oldest_nonremovable_xid. + */ + if (XLogRecPtrIsInvalid(data.remote_lsn)) + elog(ERROR, "cannot get the latest WAL position from the publisher"); + + maybe_advance_nonremovable_xid(&data, true); + + UpdateWorkerStats(last_received, data.reply_time, false); + } /* other message types are purposefully ignored */ MemoryContextReset(ApplyMessageContext); @@ -3717,6 +3821,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received) /* confirm all writes so far */ send_feedback(last_received, false, false); + /* Reset the timestamp if no message was received */ + data.last_recv_time = 0; + + maybe_advance_nonremovable_xid(&data, false); + if (!in_remote_transaction && !in_streamed_transaction) { /* @@ -3751,6 +3860,13 @@ LogicalRepApplyLoop(XLogRecPtr last_received) else wait_time = NAPTIME_PER_CYCLE; + /* + * Ensure to wake up when it's possible to attempt to advance the + * non-removable transaction ID. + */ + if (data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval) + wait_time = Min(wait_time, data.xid_advance_interval); + rc = WaitLatchOrSocket(MyLatch, WL_SOCKET_READABLE | WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH, @@ -3814,6 +3930,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received) send_feedback(last_received, requestReply, requestReply); + maybe_advance_nonremovable_xid(&data, false); + /* * Force reporting to ensure long idle periods don't lead to * arbitrarily delayed stats. Stats can only be reported outside @@ -3849,7 +3967,6 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply) static XLogRecPtr last_recvpos = InvalidXLogRecPtr; static XLogRecPtr last_writepos = InvalidXLogRecPtr; - static XLogRecPtr last_flushpos = InvalidXLogRecPtr; XLogRecPtr writepos; XLogRecPtr flushpos; @@ -3927,6 +4044,378 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply) last_flushpos = flushpos; } +/* + * Attempt to advance the non-removable transaction ID. + * + * The oldest_nonremovable_xid is maintained in shared memory to prevent dead + * rows from being removed prematurely when the apply worker still needs them + * to detect update_deleted conflicts. + * + * The non-removable transaction ID is advanced to the oldest running + * transaction ID once all concurrent transactions on the publisher have been + * applied and flushed locally. The process involves: + * + * - RCI_GET_CANDIDATE_XID: + * Call GetOldestActiveTransactionId() to take oldestRunningXid as the + * candidate xid. + * + * - RCI_REQUEST_PUBLISHER_STATUS: + * Send a message to the walsender requesting the publisher status, which + * includes the latest WAL write position and information about transactions + * that are in the commit phase. + * + * - RCI_WAIT_FOR_PUBLISHER_STATUS: + * Wait for the status from the walsender. After receiving the first status, + * do not proceed if there are concurrent remote transactions that are still + * in the commit phase. These transactions might have been assigned an + * earlier commit timestamp but have not yet written the commit WAL record. + * Continue to request the publisher status (RCI_REQUEST_PUBLISHER_STATUS) + * until all these transactions have completed. + * + * - RCI_WAIT_FOR_LOCAL_FLUSH: + * Advance the non-removable transaction ID if the current flush location has + * reached or surpassed the last received WAL position. + * + * The overall state progression is: GET_CANDIDATE_XID -> + * REQUEST_PUBLISHER_STATUS -> WAIT_FOR_PUBLISHER_STATUS -> (loop to + * REQUEST_PUBLISHER_STATUS till concurrent remote transactions end) -> + * WAIT_FOR_LOCAL_FLUSH -> loop back to GET_CANDIDATE_XID. + * + * Retaining the dead tuples for this period is sufficient for ensuring + * eventual consistency using last-update-wins strategy, as dead tuples are + * useful for detecting conflicts only during the application of concurrent + * transactions from remote nodes. After applying and flushing all remote + * transactions that occurred concurrently with the tuple DELETE, any + * subsequent UPDATE from a remote node should have a later timestamp. In such + * cases, it is acceptable to detect an update_missing scenario and convert the + * UPDATE to an INSERT when applying it. But, for concurrent remote + * transactions with earlier timestamps than the DELETE, detecting + * update_deleted is necessary, as the UPDATEs in remote transactions should be + * ignored if their timestamp is earlier than that of the dead tuples. + * + * Note that advancing the non-removable transaction ID is not supported if the + * publisher is also a physical standby. This is because the logical walsender + * on the standby can only get the WAL replay position but there may be more + * WALs that are being replicated from the primary and those WALs could have + * earlier commit timestamp. + * + * XXX It might seem feasible to track the latest commit timestamp on the + * publisher and send the WAL position once the timestamp exceeds that on the + * subscriber. However, commit timestamps can regress since a commit with a + * later LSN is not guaranteed to have a later timestamp than those with + * earlier LSNs. + */ +static void +maybe_advance_nonremovable_xid(RetainConflictInfoData *data, + bool status_received) +{ + /* + * It is sufficient to manage non-removable transaction ID for a + * subscription by the main apply worker to detect update_deleted conflict + * even for table sync or parallel apply workers. + */ + if (!am_leader_apply_worker()) + return; + + switch (data->phase) + { + case RCI_GET_CANDIDATE_XID: + get_candidate_xid(data); + break; + case RCI_REQUEST_PUBLISHER_STATUS: + request_publisher_status(data); + break; + case RCI_WAIT_FOR_PUBLISHER_STATUS: + wait_for_publisher_status(data, status_received); + break; + case RCI_WAIT_FOR_LOCAL_FLUSH: + wait_for_local_flush(data); + break; + } +} + +/* + * Workhorse for the RCI_GET_CANDIDATE_XID phase. + */ +static void +get_candidate_xid(RetainConflictInfoData *data) +{ + TransactionId oldest_running_xid; + FullTransactionId next_full_xid; + FullTransactionId full_xid; + TimestampTz now; + + /* + * Use last_recv_time when applying changes in the loop to avoid + * unnecessary system time retrieval. If last_recv_time is not available, + * obtain the current timestamp. + */ + now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp(); + + /* + * Compute the candidate_xid and request the publisher status at most once + * per xid_advance_interval. Refer to adjust_xid_advance_interval() for + * details on how this value is dynamically adjusted. This is to avoid + * using CPU and network resources without making much progress. + */ + if (!TimestampDifferenceExceeds(data->candidate_xid_time, now, + data->xid_advance_interval)) + return; + + data->candidate_xid_time = now; + + oldest_running_xid = GetOldestActiveTransactionId(); + next_full_xid = ReadNextFullTransactionId(); + + /* + * Compute FullTransactionId for the oldest running transaction ID. This + * handles the case where transaction ID wraparound has occurred. + */ + full_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid); + + /* Return if the oldest_nonremovable_xid cannot be advanced */ + if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid, + full_xid)) + { + adjust_xid_advance_interval(data, false); + return; + } + + adjust_xid_advance_interval(data, true); + + data->candidate_xid = full_xid; + data->phase = RCI_REQUEST_PUBLISHER_STATUS; + + /* process the next phase */ + maybe_advance_nonremovable_xid(data, false); +} + +/* + * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase. + */ +static void +request_publisher_status(RetainConflictInfoData *data) +{ + static StringInfo request_message = NULL; + + if (!request_message) + { + MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext); + + request_message = makeStringInfo(); + MemoryContextSwitchTo(oldctx); + } + else + resetStringInfo(request_message); + + /* + * Send the current time to update the remote walsender's latest reply + * message received time. + */ + pq_sendbyte(request_message, 'p'); + pq_sendint64(request_message, GetCurrentTimestamp()); + + elog(DEBUG2, "sending publisher status request message"); + + /* Send a request for the publisher status */ + walrcv_send(LogRepWorkerWalRcvConn, + request_message->data, request_message->len); + + data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS; + + /* + * Skip calling maybe_advance_nonremovable_xid() since further transition + * is possible only once we receive the publisher status message. + */ +} + +/* + * Workhorse for the RCI_WAIT_FOR_PUBLISHER_STATUS phase. + */ +static void +wait_for_publisher_status(RetainConflictInfoData *data, bool status_received) +{ + FullTransactionId remote_full_xid; + FullTransactionId remote_next_full_xid; + + /* + * Return if we have requested but not yet received the publisher status. + */ + if (!status_received) + return; + + remote_next_full_xid = FullTransactionIdFromEpochAndXid(data->remote_epoch, + data->remote_nextxid); + + if (!FullTransactionIdIsValid(data->last_phase_at)) + data->last_phase_at = remote_next_full_xid; + + /* + * Compute FullTransactionId for the remote oldest committing transaction + * ID. This handles the case where transaction ID wraparound has occurred. + */ + remote_full_xid = FullTransactionIdFromAllowableAt(remote_next_full_xid, + data->remote_oldestxid); + + /* + * Check if all remote concurrent transactions that were active at the + * first status request have now completed. If completed, proceed to the + * next phase; otherwise, continue checking the publisher status until + * these transactions finish. + */ + if (FullTransactionIdPrecedesOrEquals(data->last_phase_at, + remote_full_xid)) + data->phase = RCI_WAIT_FOR_LOCAL_FLUSH; + else + data->phase = RCI_REQUEST_PUBLISHER_STATUS; + + /* process the next phase */ + maybe_advance_nonremovable_xid(data, false); +} + +/* + * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase. + */ +static void +wait_for_local_flush(RetainConflictInfoData *data) +{ + Assert(!XLogRecPtrIsInvalid(data->remote_lsn) && + FullTransactionIdIsValid(data->candidate_xid)); + + /* + * We expect the publisher and subscriber clocks to be in sync using time + * sync service like NTP. Otherwise, we will advance this worker's + * oldest_nonremovable_xid prematurely, leading to the removal of rows + * required to detect update_delete conflict. + * + * XXX Consider waiting for the publisher's clock to catch up with the + * subscriber's before proceeding to the next phase. + */ + if (TimestampDifferenceExceeds(data->reply_time, + data->candidate_xid_time, 0)) + ereport(ERROR, + errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("oldest_nonremovable_xid transaction ID may be advanced prematurely"), + errdetail("The clock on the publisher is behind that of the subscriber.")); + + /* + * Do not attempt to advance the non-removable transaction ID when table + * sync is in progress. During this time, changes from a single + * transaction may be applied by multiple table sync workers corresponding + * to the target tables. So, it's necessary for all table sync workers to + * apply and flush the corresponding changes before advancing the + * transaction ID, otherwise, dead tuples that are still needed for + * conflict detection in table sync workers could be removed prematurely. + * However, confirming the apply and flush progress across all table sync + * workers is complex and not worth the effort, so we simply return if not + * all tables are in the READY state. + * + * It is safe to add new tables with initial states to the subscription + * after this check because any changes applied to these tables should + * have a WAL position greater than the data->remote_lsn. + */ + if (!AllTablesyncsReady()) + return; + + /* + * Update and check the remote flush position if we are applying changes + * in a loop. This is done at most once per WalWriterDelay to avoid + * performing costy operations in get_flush_position() too frequently + * during change application. + */ + if (data->last_recv_time && + TimestampDifferenceExceeds(data->flushpos_update_time, + data->last_recv_time, WalWriterDelay)) + { + XLogRecPtr writepos; + XLogRecPtr flushpos; + bool have_pending_txes; + + /* Fetch the latest remote flush position */ + get_flush_position(&writepos, &flushpos, &have_pending_txes); + + if (flushpos > last_flushpos) + last_flushpos = flushpos; + + data->flushpos_update_time = data->last_recv_time; + } + + /* Return to wait for the changes to be applied */ + if (last_flushpos < data->remote_lsn) + return; + + /* + * Reaching here means the remote WAL position has been received, and all + * transactions up to that position on the publisher have been applied and + * flushed locally. So, we can advance the non-removable transaction ID. + */ + SpinLockAcquire(&MyLogicalRepWorker->relmutex); + MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid; + SpinLockRelease(&MyLogicalRepWorker->relmutex); + + elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u", + LSN_FORMAT_ARGS(data->remote_lsn), + XidFromFullTransactionId(data->candidate_xid)); + + /* + * Reset all data fields except those used to determine the timing for the + * next round of transaction ID advancement. + */ + data->phase = RCI_GET_CANDIDATE_XID; + data->remote_lsn = InvalidXLogRecPtr; + data->remote_oldestxid = InvalidTransactionId; + data->remote_nextxid = InvalidTransactionId; + data->remote_epoch = 0; + data->last_phase_at = InvalidFullTransactionId; + data->candidate_xid = InvalidFullTransactionId; + data->reply_time = 0; + data->flushpos_update_time = 0; + + /* process the next phase */ + maybe_advance_nonremovable_xid(data, false); +} + +/* + * Adjust the interval for advancing non-removable transaction IDs. + * + * We double the interval to try advancing the non-removable transaction IDs + * if there is no activity on the node. The maximum value of the interval is + * capped by wal_receiver_status_interval if it is not zero, otherwise to a + * 3 minutes which should be sufficient to avoid using CPU or network + * resources without much benefit. + * + * The interval is reset to a minimum value of 100ms once there is some + * activity on the node. + * + * XXX The use of wal_receiver_status_interval is a bit arbitrary so we can + * consider the other interval or a separate GUC if the need arises. + */ +static void +adjust_xid_advance_interval(RetainConflictInfoData *data, bool new_xid_found) +{ + if (!new_xid_found && data->xid_advance_interval) + { + int max_interval = wal_receiver_status_interval + ? wal_receiver_status_interval * 1000 + : MAX_XID_ADVANCE_INTERVAL; + + /* + * No new transaction ID has been assigned since the last check, so + * double the interval, but not beyond the maximum allowable value. + */ + data->xid_advance_interval = Min(data->xid_advance_interval * 2, + max_interval); + } + else + { + /* + * A new transaction ID was found or the interval is not yet + * initialized, so set the interval to the minimum value. + */ + data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL; + } +} + /* * Exit routine for apply workers due to subscription parameter changes. */ diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c index 9fa8beb6103d..69dc382d63a6 100644 --- a/src/backend/replication/walsender.c +++ b/src/backend/replication/walsender.c @@ -84,6 +84,7 @@ #include "storage/ipc.h" #include "storage/pmsignal.h" #include "storage/proc.h" +#include "storage/procarray.h" #include "tcop/dest.h" #include "tcop/tcopprot.h" #include "utils/acl.h" @@ -258,6 +259,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd); static void ProcessStandbyMessage(void); static void ProcessStandbyReplyMessage(void); static void ProcessStandbyHSFeedbackMessage(void); +static void ProcessStandbyPSRequestMessage(void); static void ProcessRepliesIfAny(void); static void ProcessPendingWrites(void); static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr); @@ -2355,6 +2357,10 @@ ProcessStandbyMessage(void) ProcessStandbyHSFeedbackMessage(); break; + case 'p': + ProcessStandbyPSRequestMessage(); + break; + default: ereport(COMMERROR, (errcode(ERRCODE_PROTOCOL_VIOLATION), @@ -2701,6 +2707,54 @@ ProcessStandbyHSFeedbackMessage(void) } } +/* + * Process the request for a primary status update message. + */ +static void +ProcessStandbyPSRequestMessage(void) +{ + XLogRecPtr lsn = InvalidXLogRecPtr; + TransactionId oldestXidInCommit; + FullTransactionId nextFullXid; + WalSnd *walsnd = MyWalSnd; + TimestampTz replyTime; + + /* + * This shouldn't happen because we don't support getting primary status + * message from standby. + */ + if (RecoveryInProgress()) + elog(ERROR, "the primary status is unavailable during recovery"); + + replyTime = pq_getmsgint64(&reply_message); + + /* + * Update shared state for this WalSender process based on reply data from + * standby. + */ + SpinLockAcquire(&walsnd->mutex); + walsnd->replyTime = replyTime; + SpinLockRelease(&walsnd->mutex); + + oldestXidInCommit = GetOldestTransactionIdInCommit(); + nextFullXid = ReadNextFullTransactionId(); + lsn = GetXLogWriteRecPtr(); + + elog(DEBUG2, "sending primary status"); + + /* construct the message... */ + resetStringInfo(&output_message); + pq_sendbyte(&output_message, 's'); + pq_sendint64(&output_message, lsn); + pq_sendint32(&output_message, oldestXidInCommit); + pq_sendint32(&output_message, XidFromFullTransactionId(nextFullXid)); + pq_sendint32(&output_message, EpochFromFullTransactionId(nextFullXid)); + pq_sendint64(&output_message, GetCurrentTimestamp()); + + /* ... and send it wrapped in CopyData */ + pq_putmessage_noblock('d', output_message.data, output_message.len); +} + /* * Compute how long send/receive loops should sleep. * diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c index e5b945a9ee39..19dd8d7a8b79 100644 --- a/src/backend/storage/ipc/procarray.c +++ b/src/backend/storage/ipc/procarray.c @@ -2925,6 +2925,65 @@ GetOldestActiveTransactionId(void) return oldestRunningXid; } + +/* + * GetOldestTransactionIdInCommit() + * + * Similar to GetOldestActiveTransactionId but returns the oldest transaction ID + * that is currently in the commit phase. + */ +TransactionId +GetOldestTransactionIdInCommit(void) +{ + ProcArrayStruct *arrayP = procArray; + TransactionId *other_xids = ProcGlobal->xids; + TransactionId oldestXidInCommit; + int index; + + Assert(!RecoveryInProgress()); + + /* + * Read nextXid, as the upper bound of what's still active. + * + * Reading a TransactionId is atomic, but we must grab the lock to make + * sure that all XIDs < nextXid are already present in the proc array (or + * have already completed), when we spin over it. + */ + LWLockAcquire(XidGenLock, LW_SHARED); + oldestXidInCommit = XidFromFullTransactionId(TransamVariables->nextXid); + LWLockRelease(XidGenLock); + + /* + * Spin over procArray collecting all xids and subxids. + */ + LWLockAcquire(ProcArrayLock, LW_SHARED); + for (index = 0; index < arrayP->numProcs; index++) + { + TransactionId xid; + int pgprocno = arrayP->pgprocnos[index]; + PGPROC *proc = &allProcs[pgprocno]; + + /* Fetch xid just once - see GetNewTransactionId */ + xid = UINT32_ACCESS_ONCE(other_xids[index]); + + if (!TransactionIdIsNormal(xid)) + continue; + + if ((proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) != 0 && + TransactionIdPrecedes(xid, oldestXidInCommit)) + oldestXidInCommit = xid; + + /* + * Top-level XID of a transaction is always less than any of its + * subxids, so we don't need to check if any of the subxids are + * smaller than oldestXidInCommit + */ + } + LWLockRelease(ProcArrayLock); + + return oldestXidInCommit; +} + /* * GetOldestSafeDecodingTransactionId -- lowest xid not affected by vacuum * diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h index 30b2775952c3..b09486017f4a 100644 --- a/src/include/replication/worker_internal.h +++ b/src/include/replication/worker_internal.h @@ -86,6 +86,23 @@ typedef struct LogicalRepWorker /* Indicates whether apply can be performed in parallel. */ bool parallel_apply; + /* + * The changes made by this and later transactions shouldn't be removed. + * This allows the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * The logical replication launcher manages an internal replication slot + * named "pg_conflict_detection". It asynchronously collects this ID to + * decide when to advance the xmin value of the slot. + * + * It's necessary to use FullTransactionId here to mitigate potential race + * conditions. Such scenarios might occur if the replication slot is not + * yet created by the launcher while the apply worker has already + * initialized this field. During this period, a transaction ID wraparound + * could falsely make this ID appear as if it originates from the future. + */ + FullTransactionId oldest_nonremovable_xid; + /* Stats. */ XLogRecPtr last_lsn; TimestampTz last_send_time; diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h index 9f9b3fcfbf1d..65ecf3280fb7 100644 --- a/src/include/storage/proc.h +++ b/src/include/storage/proc.h @@ -130,9 +130,17 @@ extern PGDLLIMPORT int FastPathLockGroupsPerBackend; * the checkpoint are actually destroyed on disk. Replay can cope with a file * or block that doesn't exist, but not with a block that has the wrong * contents. + * + * Setting DELAY_CHKPT_IN_COMMIT is similar to setting DELAY_CHKPT_START, but + * it explicitly indicates that the reason for delaying the checkpoint is due + * to a transaction being within a critical commit section. We need this to + * ensure all such transactions are finished before we allow the logical + * replication client to advance its xid which is used to hold back dead rows + * for conflict detection. */ #define DELAY_CHKPT_START (1<<0) #define DELAY_CHKPT_COMPLETE (1<<1) +#define DELAY_CHKPT_IN_COMMIT (DELAY_CHKPT_START | 1<<2) typedef enum { diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h index ef0b733ebe8b..bd8b17a6d0d3 100644 --- a/src/include/storage/procarray.h +++ b/src/include/storage/procarray.h @@ -57,6 +57,7 @@ extern bool TransactionIdIsActive(TransactionId xid); extern TransactionId GetOldestNonRemovableTransactionId(Relation rel); extern TransactionId GetOldestTransactionIdConsideredRunning(void); extern TransactionId GetOldestActiveTransactionId(void); +extern TransactionId GetOldestTransactionIdInCommit(void); extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly); extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin); diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index e5879e00dffe..ecf0c2bd7010 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -2557,6 +2557,8 @@ RestrictInfo Result ResultRelInfo ResultState +RetainConflictInfoData +RetainConflictInfoPhase ReturnSetInfo ReturnStmt ReturningClause From 9b44a50baafddd3b5439bff7ac748a9cbd20f7a8 Mon Sep 17 00:00:00 2001 From: Hou Zhijie Date: Thu, 26 Sep 2024 12:11:34 +0800 Subject: [PATCH 2/7] Maintain the replication slot in logical launcher to retain dead tuples This patch enables the logical replication launcher to create and maintain a replication slot named pg_conflict_detection. The launcher periodically collects the oldest_nonremovable_xid from all apply workers. It then computes the minimum transaction ID and advances the xmin value of the replication slot if it precedes the computed value. The interval for updating the slot (nap time) is dynamically adjusted based on the activity of the apply workers. The launcher waits for a certain period before performing the next update, with the duration varying depending on whether the xmin value of the replication slot was updated during the last cycle. --- doc/src/sgml/config.sgml | 2 + doc/src/sgml/func.sgml | 14 +- doc/src/sgml/protocol.sgml | 2 + doc/src/sgml/ref/create_subscription.sgml | 4 +- src/backend/access/transam/xlogrecovery.c | 2 +- src/backend/commands/subscriptioncmds.c | 2 +- src/backend/replication/logical/launcher.c | 180 +++++++++++++++++- .../replication/logical/reorderbuffer.c | 2 +- src/backend/replication/logical/worker.c | 3 + src/backend/replication/slot.c | 34 +++- src/include/replication/logicallauncher.h | 1 + src/include/replication/slot.h | 11 +- 12 files changed, 245 insertions(+), 12 deletions(-) diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml index fd6e3e028907..54afce86d766 100644 --- a/doc/src/sgml/config.sgml +++ b/doc/src/sgml/config.sgml @@ -4945,6 +4945,8 @@ ANY num_sync ( The name of the slot to create. Must be a valid replication slot name (see ). + The name cannot be pg_conflict_detection, as it + is reserved for logical replication conflict detection. diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml index 57dec28a5df6..eec85cde8803 100644 --- a/doc/src/sgml/ref/create_subscription.sgml +++ b/doc/src/sgml/ref/create_subscription.sgml @@ -169,7 +169,9 @@ CREATE SUBSCRIPTION subscription_name Name of the publisher's replication slot to use. The default is - to use the name of the subscription for the slot name. + to use the name of the subscription for the slot name. The name cannot + be pg_conflict_detection, as it is reserved for + logical replication conflict detection. diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c index 6ce979f2d8bc..2dcda37bc770 100644 --- a/src/backend/access/transam/xlogrecovery.c +++ b/src/backend/access/transam/xlogrecovery.c @@ -4760,7 +4760,7 @@ bool check_primary_slot_name(char **newval, void **extra, GucSource source) { if (*newval && strcmp(*newval, "") != 0 && - !ReplicationSlotValidateName(*newval, WARNING)) + !ReplicationSlotValidateName(*newval, false, WARNING)) return false; return true; diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c index 4aec73bcc6bb..46d4e65da970 100644 --- a/src/backend/commands/subscriptioncmds.c +++ b/src/backend/commands/subscriptioncmds.c @@ -210,7 +210,7 @@ parse_subscription_options(ParseState *pstate, List *stmt_options, if (strcmp(opts->slot_name, "none") == 0) opts->slot_name = NULL; else - ReplicationSlotValidateName(opts->slot_name, ERROR); + ReplicationSlotValidateName(opts->slot_name, false, ERROR); } else if (IsSet(supported_opts, SUBOPT_COPY_DATA) && strcmp(defel->defname, "copy_data") == 0) diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c index 8e42787a4265..8cef44608487 100644 --- a/src/backend/replication/logical/launcher.c +++ b/src/backend/replication/logical/launcher.c @@ -32,6 +32,7 @@ #include "postmaster/interrupt.h" #include "replication/logicallauncher.h" #include "replication/origin.h" +#include "replication/slot.h" #include "replication/walreceiver.h" #include "replication/worker_internal.h" #include "storage/ipc.h" @@ -91,7 +92,6 @@ static dshash_table *last_start_times = NULL; static bool on_commit_launcher_wakeup = false; -static void ApplyLauncherWakeup(void); static void logicalrep_launcher_onexit(int code, Datum arg); static void logicalrep_worker_onexit(int code, Datum arg); static void logicalrep_worker_detach(void); @@ -100,6 +100,9 @@ static int logicalrep_pa_worker_count(Oid subid); static void logicalrep_launcher_attach_dshmem(void); static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time); static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid); +static void create_conflict_slot_if_not_exists(void); +static void advance_conflict_slot_xmin(FullTransactionId new_xmin); +static void drop_conflict_slot_if_exists(void); /* @@ -1106,7 +1109,10 @@ ApplyLauncherWakeupAtCommit(void) on_commit_launcher_wakeup = true; } -static void +/* + * Wakeup the launcher immediately. + */ +void ApplyLauncherWakeup(void) { if (LogicalRepCtx->launcher_pid != 0) @@ -1119,6 +1125,8 @@ ApplyLauncherWakeup(void) void ApplyLauncherMain(Datum main_arg) { + bool slot_maybe_exist = true; + ereport(DEBUG1, (errmsg_internal("logical replication launcher started"))); @@ -1147,6 +1155,8 @@ ApplyLauncherMain(Datum main_arg) MemoryContext subctx; MemoryContext oldctx; long wait_time = DEFAULT_NAPTIME_PER_CYCLE; + bool can_advance_xmin = true; + FullTransactionId xmin = InvalidFullTransactionId; CHECK_FOR_INTERRUPTS(); @@ -1166,15 +1176,56 @@ ApplyLauncherMain(Datum main_arg) TimestampTz now; long elapsed; + /* + * Create the conflict slot before starting the worker to prevent + * it from unnecessarily maintaining its oldest_nonremovable_xid. + */ + create_conflict_slot_if_not_exists(); + if (!sub->enabled) + { + can_advance_xmin = false; continue; + } LWLockAcquire(LogicalRepWorkerLock, LW_SHARED); w = logicalrep_worker_find(sub->oid, InvalidOid, false); LWLockRelease(LogicalRepWorkerLock); if (w != NULL) + { + /* + * Collect non-removable transaction IDs from all apply + * workers to determine the xmin for advancing the replication + * slot used in conflict detection. + */ + if (can_advance_xmin) + { + FullTransactionId nonremovable_xid; + + SpinLockAcquire(&w->relmutex); + nonremovable_xid = w->oldest_nonremovable_xid; + SpinLockRelease(&w->relmutex); + + /* + * Stop advancing xmin if an invalid non-removable + * transaction ID is found, otherwise update xmin. + */ + if (!FullTransactionIdIsValid(nonremovable_xid)) + can_advance_xmin = false; + else if (!FullTransactionIdIsValid(xmin) || + FullTransactionIdPrecedes(nonremovable_xid, xmin)) + xmin = nonremovable_xid; + } + continue; /* worker is running already */ + } + + /* + * The worker has not yet started, so there is no valid + * non-removable transaction ID available for advancement. + */ + can_advance_xmin = false; /* * If the worker is eligible to start now, launch it. Otherwise, @@ -1207,6 +1258,27 @@ ApplyLauncherMain(Datum main_arg) } } + /* + * Maintain the xmin value of the replication slot for conflict + * detection if needed. + */ + if (sublist) + { + if (can_advance_xmin) + advance_conflict_slot_xmin(xmin); + + slot_maybe_exist = true; + } + + /* + * Drop the slot if we're no longer retaining dead tuples. + */ + else if (slot_maybe_exist) + { + drop_conflict_slot_if_exists(); + slot_maybe_exist = false; + } + /* Switch back to original memory context. */ MemoryContextSwitchTo(oldctx); /* Clean the temporary memory. */ @@ -1234,6 +1306,110 @@ ApplyLauncherMain(Datum main_arg) /* Not reachable */ } +/* + * Create and acquire the replication slot used to retain dead tuples for + * conflict detection, if not yet. + */ +static void +create_conflict_slot_if_not_exists(void) +{ + TransactionId xmin_horizon; + + /* Exit early if the replication slot is already created and acquired */ + if (MyReplicationSlot) + return; + + /* If the replication slot exists, acquire it and exit */ + if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true)) + { + ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false); + return; + } + + ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, + RS_PERSISTENT, false, false, false); + + LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE); + + xmin_horizon = GetOldestSafeDecodingTransactionId(false); + + SpinLockAcquire(&MyReplicationSlot->mutex); + MyReplicationSlot->effective_xmin = xmin_horizon; + MyReplicationSlot->data.xmin = xmin_horizon; + SpinLockRelease(&MyReplicationSlot->mutex); + + ReplicationSlotsComputeRequiredXmin(true); + + LWLockRelease(ProcArrayLock); + + /* Write this slot to disk */ + ReplicationSlotMarkDirty(); + ReplicationSlotSave(); +} + +/* + * Attempt to advance the xmin value of the replication slot used to retain + * dead tuples for conflict detection. + */ +static void +advance_conflict_slot_xmin(FullTransactionId new_xmin) +{ + FullTransactionId full_xmin; + FullTransactionId next_full_xid; + + Assert(MyReplicationSlot); + Assert(FullTransactionIdIsValid(new_xmin)); + + next_full_xid = ReadNextFullTransactionId(); + + /* + * Compute FullTransactionId for the current xmin. This handles the case + * where transaction ID wraparound has occurred. + */ + full_xmin = FullTransactionIdFromAllowableAt(next_full_xid, + MyReplicationSlot->data.xmin); + + if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin)) + return; + + SpinLockAcquire(&MyReplicationSlot->mutex); + MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin); + SpinLockRelease(&MyReplicationSlot->mutex); + + /* first write new xmin to disk, so we know what's up after a crash */ + + ReplicationSlotMarkDirty(); + ReplicationSlotSave(); + elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin); + + /* + * Now the new xmin is safely on disk, we can let the global value + * advance. We do not take ProcArrayLock or similar since we only advance + * xmin here and there's not much harm done by a concurrent computation + * missing that. + */ + SpinLockAcquire(&MyReplicationSlot->mutex); + MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin; + SpinLockRelease(&MyReplicationSlot->mutex); + + ReplicationSlotsComputeRequiredXmin(false); + + return; +} + +/* + * Drop the replication slot used to retain dead tuples for conflict detection, + * if it exists. + */ +static void +drop_conflict_slot_if_exists(void) +{ + if (MyReplicationSlot) + ReplicationSlotDropAcquired(); + else if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true)) + ReplicationSlotDrop(CONFLICT_DETECTION_SLOT, true); +} + /* * Is current process the logical replication launcher? */ diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c index 676551118753..85239f6c3167 100644 --- a/src/backend/replication/logical/reorderbuffer.c +++ b/src/backend/replication/logical/reorderbuffer.c @@ -4787,7 +4787,7 @@ StartupReorderBuffer(void) continue; /* if it cannot be a slot, skip the directory */ - if (!ReplicationSlotValidateName(logical_de->d_name, DEBUG2)) + if (!ReplicationSlotValidateName(logical_de->d_name, true, DEBUG2)) continue; /* diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c index ffbd4e3a02c7..4922104b018f 100644 --- a/src/backend/replication/logical/worker.c +++ b/src/backend/replication/logical/worker.c @@ -4357,6 +4357,9 @@ wait_for_local_flush(RetainConflictInfoData *data) LSN_FORMAT_ARGS(data->remote_lsn), XidFromFullTransactionId(data->candidate_xid)); + /* Notify launcher to update the xmin of the conflict slot */ + ApplyLauncherWakeup(); + /* * Reset all data fields except those used to determine the timing for the * next round of transaction ID advancement. diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c index 600b87fa9cb6..668279cb4e8e 100644 --- a/src/backend/replication/slot.c +++ b/src/backend/replication/slot.c @@ -47,6 +47,7 @@ #include "miscadmin.h" #include "pgstat.h" #include "postmaster/interrupt.h" +#include "replication/logicallauncher.h" #include "replication/slotsync.h" #include "replication/slot.h" #include "replication/walsender_private.h" @@ -172,6 +173,7 @@ static SyncStandbySlotsConfigData *synchronized_standby_slots_config; static XLogRecPtr ss_oldest_flush_lsn = InvalidXLogRecPtr; static void ReplicationSlotShmemExit(int code, Datum arg); +static bool IsReservedSlotName(const char *name); static void ReplicationSlotDropPtr(ReplicationSlot *slot); /* internal persistency functions */ @@ -258,13 +260,17 @@ ReplicationSlotShmemExit(int code, Datum arg) /* * Check whether the passed slot name is valid and report errors at elevel. * + * An error will be reported for a reserved replication slot name if + * allow_reserved_name is set to false. + * * Slot names may consist out of [a-z0-9_]{1,NAMEDATALEN-1} which should allow * the name to be used as a directory name on every supported OS. * * Returns whether the directory name is valid or not if elevel < ERROR. */ bool -ReplicationSlotValidateName(const char *name, int elevel) +ReplicationSlotValidateName(const char *name, bool allow_reserved_name, + int elevel) { const char *cp; @@ -300,9 +306,29 @@ ReplicationSlotValidateName(const char *name, int elevel) return false; } } + + if (!allow_reserved_name && IsReservedSlotName(name)) + { + ereport(elevel, + errcode(ERRCODE_RESERVED_NAME), + errmsg("replication slot name \"%s\" is reserved", + name)); + + return false; + } + return true; } +/* + * Return true if the replication slot name is "pg_conflict_detection". + */ +static bool +IsReservedSlotName(const char *name) +{ + return (strcmp(name, CONFLICT_DETECTION_SLOT) == 0); +} + /* * Create a new replication slot and mark it as used by this backend. * @@ -330,7 +356,11 @@ ReplicationSlotCreate(const char *name, bool db_specific, Assert(MyReplicationSlot == NULL); - ReplicationSlotValidateName(name, ERROR); + /* + * The logical launcher might be creating an internal slot, so using a + * reserved name is allowed in this case. + */ + ReplicationSlotValidateName(name, IsLogicalLauncher(), ERROR); if (failover) { diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h index 82b202f33051..7b29f1814db3 100644 --- a/src/include/replication/logicallauncher.h +++ b/src/include/replication/logicallauncher.h @@ -25,6 +25,7 @@ extern void ApplyLauncherShmemInit(void); extern void ApplyLauncherForgetWorkerStartTime(Oid subid); extern void ApplyLauncherWakeupAtCommit(void); +extern void ApplyLauncherWakeup(void); extern void AtEOXact_ApplyLauncher(bool isCommit); extern bool IsLogicalLauncher(void); diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h index eb0b93b11141..e03e123a2ff9 100644 --- a/src/include/replication/slot.h +++ b/src/include/replication/slot.h @@ -20,6 +20,13 @@ /* directory to store replication slot data in */ #define PG_REPLSLOT_DIR "pg_replslot" +/* + * The reserved name for a replication slot used to retain dead tuples for + * conflict detection in logical replication. See + * maybe_advance_nonremovable_xid() for detail. + */ +#define CONFLICT_DETECTION_SLOT "pg_conflict_detection" + /* * Behaviour of replication slots, upon release or crash. * @@ -284,7 +291,9 @@ extern void ReplicationSlotMarkDirty(void); /* misc stuff */ extern void ReplicationSlotInitialize(void); -extern bool ReplicationSlotValidateName(const char *name, int elevel); +extern bool ReplicationSlotValidateName(const char *name, + bool allow_reserved_name, + int elevel); extern void ReplicationSlotReserveWal(void); extern void ReplicationSlotsComputeRequiredXmin(bool already_locked); extern void ReplicationSlotsComputeRequiredLSN(void); From fd28cc279fcdc222ad8ee91a4557ac1528075171 Mon Sep 17 00:00:00 2001 From: Zhijie Hou Date: Thu, 24 Apr 2025 20:38:20 +0800 Subject: [PATCH 3/7] Add a retain_conflict_info option to subscriptions This patch adds a subscription option allowing users to specify whether information on the subscriber, which is useful for detecting update_deleted conflicts, should be retained. The default setting is false. If set to true, the detection of update_deleted will be enabled, and an additional replication slot named pg_conflict_detection will be created on the subscriber to prevent conflict information from being removed. Note that if multiple subscriptions on one node enable this option, only one replication slot will be created. The logical launcher will create and maintain a replication slot named pg_conflict_detection only if any local subscription has the retain_conflict_info option enabled. Enabling retain_conflict_info is prohibited if the publisher is currently in recovery mode (operating as a standby server). Bump catalog version --- doc/src/sgml/catalogs.sgml | 12 ++ doc/src/sgml/logical-replication.sgml | 3 +- doc/src/sgml/ref/alter_subscription.sgml | 12 +- doc/src/sgml/ref/create_subscription.sgml | 29 ++++ src/backend/catalog/pg_subscription.c | 1 + src/backend/catalog/system_views.sql | 3 +- src/backend/commands/subscriptioncmds.c | 151 +++++++++++++++-- src/backend/replication/logical/launcher.c | 51 ++++-- src/backend/replication/logical/worker.c | 16 +- src/bin/pg_dump/pg_dump.c | 18 ++- src/bin/pg_dump/pg_dump.h | 1 + src/bin/pg_upgrade/check.c | 52 +++++- src/bin/pg_upgrade/info.c | 25 ++- src/bin/pg_upgrade/pg_upgrade.h | 4 +- src/bin/pg_upgrade/t/004_subscription.pl | 48 ++++++ src/bin/psql/describe.c | 6 +- src/bin/psql/tab-complete.in.c | 10 +- src/include/catalog/pg_subscription.h | 5 + src/test/regress/expected/subscription.out | 178 ++++++++++++--------- src/test/regress/sql/subscription.sql | 16 ++ 20 files changed, 512 insertions(+), 129 deletions(-) diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml index cbd4e40a320b..0fb3894feb65 100644 --- a/doc/src/sgml/catalogs.sgml +++ b/doc/src/sgml/catalogs.sgml @@ -8088,6 +8088,18 @@ SCRAM-SHA-256$<iteration count>:&l + + + subretainconflictinfo bool + + + If true, the detection of is + enabled and the information (e.g., dead tuples, commit timestamps, and + origins) on the subscriber that is still useful for conflict detection + is retained. + + + subconninfo text diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml index f288c049a5c9..9fcb7c0ff73b 100644 --- a/doc/src/sgml/logical-replication.sgml +++ b/doc/src/sgml/logical-replication.sgml @@ -2408,7 +2408,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER max_replication_slots must be set to at least the number of subscriptions expected to connect, - plus some reserve for table synchronization. + plus some reserve for table synchronization and one if + retain_conflict_info is enabled. diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml index fdc648d007f1..e5415c3150d8 100644 --- a/doc/src/sgml/ref/alter_subscription.sgml +++ b/doc/src/sgml/ref/alter_subscription.sgml @@ -235,8 +235,9 @@ ALTER SUBSCRIPTION name RENAME TO < password_required, run_as_owner, origin, - failover, and - two_phase. + failover, + two_phase, and + retain_conflict_info. Only a superuser can set password_required = false. @@ -285,6 +286,13 @@ ALTER SUBSCRIPTION name RENAME TO < option is changed from true to false, the publisher will replicate the transactions again when they are committed. + + + If the retain_conflict_info + option is altered to false and no other subscription + has this option enabled, the additional replication slot that was created + to retain conflict information will be dropped. + diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml index eec85cde8803..b7ec696eb3c9 100644 --- a/doc/src/sgml/ref/create_subscription.sgml +++ b/doc/src/sgml/ref/create_subscription.sgml @@ -437,6 +437,35 @@ CREATE SUBSCRIPTION subscription_name + + + retain_conflict_info (boolean) + + + Specifies whether the information (e.g., dead tuples, commit + timestamps, and origins) on the subscriber that is still useful for + conflict detection is retained. The default is + false. If set to true, the detection of + is enabled, and an + additional replication slot named + pg_conflict_detection will be + created on the subscriber to prevent the conflict information from + being removed. + + + + Note that the information useful for conflict detection is retained + only after the creation of the additional slot. You can verify the + existence of this slot by querying pg_replication_slots.conflicting + And even if multiple subscriptions on one node enable this option, + only one replication slot will be created. + + + + This option cannot be enabled if the publisher is also a physical standby. + + + diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c index 1395032413e3..39cfae43d6f8 100644 --- a/src/backend/catalog/pg_subscription.c +++ b/src/backend/catalog/pg_subscription.c @@ -103,6 +103,7 @@ GetSubscription(Oid subid, bool missing_ok) sub->passwordrequired = subform->subpasswordrequired; sub->runasowner = subform->subrunasowner; sub->failover = subform->subfailover; + sub->retainconflictinfo = subform->subretainconflictinfo; /* Get conninfo */ datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID, diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql index 15efb02badb8..637a312e1987 100644 --- a/src/backend/catalog/system_views.sql +++ b/src/backend/catalog/system_views.sql @@ -1383,7 +1383,8 @@ REVOKE ALL ON pg_subscription FROM public; GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled, subbinary, substream, subtwophasestate, subdisableonerr, subpasswordrequired, subrunasowner, subfailover, - subslotname, subsynccommit, subpublications, suborigin) + subretainconflictinfo, subslotname, subsynccommit, + subpublications, suborigin) ON pg_subscription TO public; CREATE VIEW pg_stat_subscription_stats AS diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c index 46d4e65da970..47ba22d28b8c 100644 --- a/src/backend/commands/subscriptioncmds.c +++ b/src/backend/commands/subscriptioncmds.c @@ -14,6 +14,7 @@ #include "postgres.h" +#include "access/commit_ts.h" #include "access/htup_details.h" #include "access/table.h" #include "access/twophase.h" @@ -71,8 +72,9 @@ #define SUBOPT_PASSWORD_REQUIRED 0x00000800 #define SUBOPT_RUN_AS_OWNER 0x00001000 #define SUBOPT_FAILOVER 0x00002000 -#define SUBOPT_LSN 0x00004000 -#define SUBOPT_ORIGIN 0x00008000 +#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000 +#define SUBOPT_LSN 0x00008000 +#define SUBOPT_ORIGIN 0x00010000 /* check if the 'val' has 'bits' set */ #define IsSet(val, bits) (((val) & (bits)) == (bits)) @@ -98,6 +100,7 @@ typedef struct SubOpts bool passwordrequired; bool runasowner; bool failover; + bool retainconflictinfo; char *origin; XLogRecPtr lsn; } SubOpts; @@ -107,6 +110,8 @@ static void check_publications_origin(WalReceiverConn *wrconn, List *publications, bool copydata, char *origin, Oid *subrel_local_oids, int subrel_count, char *subname); +static void check_conflict_info_retaintion(WalReceiverConn *wrconn, + bool retain_conflict_info); static void check_duplicates_in_publist(List *publist, Datum *datums); static List *merge_publications(List *oldpublist, List *newpublist, bool addpub, const char *subname); static void ReportSlotConnectionError(List *rstates, Oid subid, char *slotname, char *err); @@ -162,6 +167,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options, opts->runasowner = false; if (IsSet(supported_opts, SUBOPT_FAILOVER)) opts->failover = false; + if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO)) + opts->retainconflictinfo = false; if (IsSet(supported_opts, SUBOPT_ORIGIN)) opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY); @@ -307,6 +314,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options, opts->specified_opts |= SUBOPT_FAILOVER; opts->failover = defGetBoolean(defel); } + else if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO) && + strcmp(defel->defname, "retain_conflict_info") == 0) + { + if (IsSet(opts->specified_opts, SUBOPT_RETAIN_CONFLICT_INFO)) + errorConflictingDefElem(defel, pstate); + + opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO; + opts->retainconflictinfo = defGetBoolean(defel); + } else if (IsSet(supported_opts, SUBOPT_ORIGIN) && strcmp(defel->defname, "origin") == 0) { @@ -563,7 +579,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt, SUBOPT_SYNCHRONOUS_COMMIT | SUBOPT_BINARY | SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT | SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED | - SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER | SUBOPT_ORIGIN); + SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER | + SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN); parse_subscription_options(pstate, stmt->options, supported_opts, &opts); /* @@ -608,6 +625,12 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt, errmsg("password_required=false is superuser-only"), errhint("Subscriptions with the password_required option set to false may only be created or modified by the superuser."))); + if (opts.retainconflictinfo && !track_commit_timestamp) + ereport(WARNING, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("information for detecting conflicts cannot be fully retained when \"%s\" is disabled", + "track_commit_timestamp")); + /* * If built with appropriate switch, whine when regression-testing * conventions for subscription names are violated. @@ -670,6 +693,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt, values[Anum_pg_subscription_subpasswordrequired - 1] = BoolGetDatum(opts.passwordrequired); values[Anum_pg_subscription_subrunasowner - 1] = BoolGetDatum(opts.runasowner); values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover); + values[Anum_pg_subscription_subretainconflictinfo - 1] = + BoolGetDatum(opts.retainconflictinfo); values[Anum_pg_subscription_subconninfo - 1] = CStringGetTextDatum(conninfo); if (opts.slot_name) @@ -724,6 +749,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt, check_publications_origin(wrconn, publications, opts.copy_data, opts.origin, NULL, 0, stmt->subname); + check_conflict_info_retaintion(wrconn, opts.retainconflictinfo); + /* * Set sync state based on if we were asked to do data copy or * not. @@ -1110,6 +1137,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool update_tuple = false; bool update_failover = false; bool update_two_phase = false; + bool retain_conflict_info = false; Subscription *sub; Form_pg_subscription form; bits32 supported_opts; @@ -1165,7 +1193,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED | SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER | - SUBOPT_ORIGIN); + SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN); parse_subscription_options(pstate, stmt->options, supported_opts, &opts); @@ -1325,6 +1353,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, replaces[Anum_pg_subscription_subfailover - 1] = true; } + if (IsSet(opts.specified_opts, SUBOPT_RETAIN_CONFLICT_INFO)) + { + values[Anum_pg_subscription_subretainconflictinfo - 1] = + BoolGetDatum(opts.retainconflictinfo); + replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true; + + if (opts.retainconflictinfo && !track_commit_timestamp) + ereport(WARNING, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("information for detecting conflicts cannot be fully retained when \"%s\" is disabled", + "track_commit_timestamp")); + + /* + * Notify the launcher to manage the replication slot for + * conflict detection. This ensures that replication slot + * is efficiently handled (created, updated, or dropped) + * in response to any configuration changes. + */ + ApplyLauncherWakeupAtCommit(); + + retain_conflict_info = opts.retainconflictinfo; + } + if (IsSet(opts.specified_opts, SUBOPT_ORIGIN)) { values[Anum_pg_subscription_suborigin - 1] = @@ -1355,6 +1406,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, ApplyLauncherWakeupAtCommit(); update_tuple = true; + + /* + * The subscription might be initially created with + * connect=false and retain_conflict_info=true, meaning the + * remote server's status may not be checked. Ensure this + * check is conducted now. + */ + retain_conflict_info = sub->retainconflictinfo; break; } @@ -1369,6 +1428,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, CStringGetTextDatum(stmt->conninfo); replaces[Anum_pg_subscription_subconninfo - 1] = true; update_tuple = true; + + /* + * Since the remote server configuration might have changed, + * perform a check to ensure it permits enabling + * retain_conflict_info. + */ + retain_conflict_info = sub->retainconflictinfo; break; case ALTER_SUBSCRIPTION_SET_PUBLICATION: @@ -1568,14 +1634,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, } /* - * Try to acquire the connection necessary for altering the slot, if - * needed. + * Try to acquire the connection necessary either for modifying the slot + * or for checking if the remote server permits enabling + * retain_conflict_info. * * This has to be at the end because otherwise if there is an error while * doing the database operations we won't be able to rollback altered * slot. */ - if (update_failover || update_two_phase) + if (update_failover || update_two_phase || retain_conflict_info) { bool must_use_password; char *err; @@ -1584,10 +1651,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, /* Load the library providing us libpq calls. */ load_file("libpqwalreceiver", false); - /* Try to connect to the publisher. */ + /* + * Try to connect to the publisher, using the new connection string if + * available. + */ must_use_password = sub->passwordrequired && !sub->ownersuperuser; - wrconn = walrcv_connect(sub->conninfo, true, true, must_use_password, - sub->name, &err); + wrconn = walrcv_connect(stmt->conninfo ? stmt->conninfo : sub->conninfo, + true, true, must_use_password, sub->name, + &err); if (!wrconn) ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE), @@ -1596,9 +1667,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, PG_TRY(); { - walrcv_alter_slot(wrconn, sub->slotname, - update_failover ? &opts.failover : NULL, - update_two_phase ? &opts.twophase : NULL); + check_conflict_info_retaintion(wrconn, retain_conflict_info); + + if (update_failover || update_two_phase) + walrcv_alter_slot(wrconn, sub->slotname, + update_failover ? &opts.failover : NULL, + update_two_phase ? &opts.twophase : NULL); } PG_FINALLY(); { @@ -2196,6 +2270,57 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications, walrcv_clear_result(res); } +/* + * Check if the publisher's status permits enabling retain_conflict_info. + * + * Enabling retain_conflict_info is not allowed if the publisher's version is + * prior to PG18 or if the publisher is in recovery (operating as a standby + * server). + * + * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed + * reasons. + */ +static void +check_conflict_info_retaintion(WalReceiverConn *wrconn, bool retain_conflict_info) +{ + WalRcvExecResult *res; + Oid RecoveryRow[1] = {BOOLOID}; + TupleTableSlot *slot; + bool isnull; + bool remote_in_recovery; + + if (!retain_conflict_info) + return; + + if (walrcv_server_version(wrconn) < 18000) + ereport(ERROR, + errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("cannot enable retain_conflict_info if the publisher is running a version earlier than PostgreSQL 18.")); + + res = walrcv_exec(wrconn, "SELECT pg_is_in_recovery()", 1, RecoveryRow); + + if (res->status != WALRCV_OK_TUPLES) + ereport(ERROR, + (errcode(ERRCODE_CONNECTION_FAILURE), + errmsg("could not obtain recovery progress from the publisher: %s", + res->err))); + + slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple); + if (!tuplestore_gettupleslot(res->tuplestore, true, false, slot)) + elog(ERROR, "failed to fetch tuple for the recovery progress"); + + remote_in_recovery = DatumGetBool(slot_getattr(slot, 1, &isnull)); + + if (remote_in_recovery) + ereport(ERROR, + errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot enable retain_conflict_info if the publisher is in recovery.")); + + ExecDropSingleTupleTableSlot(slot); + + walrcv_clear_result(res); +} + /* * Get the list of tables which belong to specified publications on the * publisher connection. diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c index 8cef44608487..07d686c6d8cd 100644 --- a/src/backend/replication/logical/launcher.c +++ b/src/backend/replication/logical/launcher.c @@ -151,6 +151,7 @@ get_subscription_list(void) sub->owner = subform->subowner; sub->enabled = subform->subenabled; sub->name = pstrdup(NameStr(subform->subname)); + sub->retainconflictinfo = subform->subretainconflictinfo; /* We don't fill fields we are not interested in. */ res = lappend(res, sub); @@ -1156,6 +1157,7 @@ ApplyLauncherMain(Datum main_arg) MemoryContext oldctx; long wait_time = DEFAULT_NAPTIME_PER_CYCLE; bool can_advance_xmin = true; + bool retain_conflict_info = false; FullTransactionId xmin = InvalidFullTransactionId; CHECK_FOR_INTERRUPTS(); @@ -1177,16 +1179,26 @@ ApplyLauncherMain(Datum main_arg) long elapsed; /* - * Create the conflict slot before starting the worker to prevent - * it from unnecessarily maintaining its oldest_nonremovable_xid. + * Create a replication slot to retain information (e.g., dead + * tuples, commit timestamps, and origins) useful for conflict + * detection if any subscription requests it. Only advance xmin + * when all such subscriptions are enabled. */ - create_conflict_slot_if_not_exists(); + if (sub->retainconflictinfo) + { + retain_conflict_info = true; + can_advance_xmin &= sub->enabled; + + /* + * Create the conflict slot before starting the worker to + * prevent it from unnecessarily maintaining its + * oldest_nonremovable_xid. + */ + create_conflict_slot_if_not_exists(); + } if (!sub->enabled) - { - can_advance_xmin = false; continue; - } LWLockAcquire(LogicalRepWorkerLock, LW_SHARED); w = logicalrep_worker_find(sub->oid, InvalidOid, false); @@ -1196,10 +1208,11 @@ ApplyLauncherMain(Datum main_arg) { /* * Collect non-removable transaction IDs from all apply - * workers to determine the xmin for advancing the replication - * slot used in conflict detection. + * workers that enabled retain_conflict_info. This determines + * the new xmin for advancing the replication slot used in + * conflict detection. */ - if (can_advance_xmin) + if (sub->retainconflictinfo && can_advance_xmin) { FullTransactionId nonremovable_xid; @@ -1225,7 +1238,8 @@ ApplyLauncherMain(Datum main_arg) * The worker has not yet started, so there is no valid * non-removable transaction ID available for advancement. */ - can_advance_xmin = false; + if (sub->retainconflictinfo) + can_advance_xmin = false; /* * If the worker is eligible to start now, launch it. Otherwise, @@ -1262,7 +1276,7 @@ ApplyLauncherMain(Datum main_arg) * Maintain the xmin value of the replication slot for conflict * detection if needed. */ - if (sublist) + if (retain_conflict_info) { if (can_advance_xmin) advance_conflict_slot_xmin(xmin); @@ -1271,7 +1285,8 @@ ApplyLauncherMain(Datum main_arg) } /* - * Drop the slot if we're no longer retaining dead tuples. + * Drop the slot if we're no longer retaining information useful for + * conflict detection */ else if (slot_maybe_exist) { @@ -1307,7 +1322,7 @@ ApplyLauncherMain(Datum main_arg) } /* - * Create and acquire the replication slot used to retain dead tuples for + * Create and acquire the replication slot used to retain information for * conflict detection, if not yet. */ static void @@ -1349,7 +1364,7 @@ create_conflict_slot_if_not_exists(void) /* * Attempt to advance the xmin value of the replication slot used to retain - * dead tuples for conflict detection. + * information useful for conflict detection. */ static void advance_conflict_slot_xmin(FullTransactionId new_xmin) @@ -1358,7 +1373,9 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin) FullTransactionId next_full_xid; Assert(MyReplicationSlot); - Assert(FullTransactionIdIsValid(new_xmin)); + + if (!FullTransactionIdIsValid(new_xmin)) + return; next_full_xid = ReadNextFullTransactionId(); @@ -1398,8 +1415,8 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin) } /* - * Drop the replication slot used to retain dead tuples for conflict detection, - * if it exists. + * Drop the replication slot used to retain information useful for conflict + * detection, if it exists. */ static void drop_conflict_slot_if_exists(void) diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c index 4922104b018f..471a95da3fb1 100644 --- a/src/backend/replication/logical/worker.c +++ b/src/backend/replication/logical/worker.c @@ -4109,6 +4109,10 @@ static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data, bool status_received) { + /* Exit early if retaining conflict information is not required */ + if (!MySubscription->retainconflictinfo) + return; + /* * It is sufficient to manage non-removable transaction ID for a * subscription by the main apply worker to detect update_deleted conflict @@ -4518,6 +4522,15 @@ maybe_reread_subscription(void) * worker won't restart if the streaming option's value is changed from * 'parallel' to any other value or the server decides not to stream the * in-progress transaction. + * + * Additionally, exit if the retain_conflict_info option is disabled. This + * is necessary to reset the oldest non-removable transaction ID and the + * state of advancement. Direct resetting could not work without a + * restart, as the worker might be in an intermediate state (e.g., waiting + * publisher status). If the option is re-enabled before the old publisher + * status is received, it could incorrectly use the old status in a new + * transaction ID advancement cycle, leading to premature advancement of + * the non-removable transaction ID. */ if (strcmp(newsub->conninfo, MySubscription->conninfo) != 0 || strcmp(newsub->name, MySubscription->name) != 0 || @@ -4527,7 +4540,8 @@ maybe_reread_subscription(void) newsub->passwordrequired != MySubscription->passwordrequired || strcmp(newsub->origin, MySubscription->origin) != 0 || newsub->owner != MySubscription->owner || - !equal(newsub->publications, MySubscription->publications)) + !equal(newsub->publications, MySubscription->publications) || + (!newsub->retainconflictinfo && MySubscription->retainconflictinfo)) { if (am_parallel_apply_worker()) ereport(LOG, diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c index e2e7975b34e0..1a34dd2ccb38 100644 --- a/src/bin/pg_dump/pg_dump.c +++ b/src/bin/pg_dump/pg_dump.c @@ -4960,6 +4960,7 @@ getSubscriptions(Archive *fout) int i_suboriginremotelsn; int i_subenabled; int i_subfailover; + int i_subretainconflictinfo; int i, ntups; @@ -5032,10 +5033,17 @@ getSubscriptions(Archive *fout) if (fout->remoteVersion >= 170000) appendPQExpBufferStr(query, - " s.subfailover\n"); + " s.subfailover,\n"); else appendPQExpBufferStr(query, - " false AS subfailover\n"); + " false AS subfailover,\n"); + + if (fout->remoteVersion >= 180000) + appendPQExpBufferStr(query, + " s.subretainconflictinfo\n"); + else + appendPQExpBufferStr(query, + " false AS subretainconflictinfo\n"); appendPQExpBufferStr(query, "FROM pg_subscription s\n"); @@ -5069,6 +5077,7 @@ getSubscriptions(Archive *fout) i_subpasswordrequired = PQfnumber(res, "subpasswordrequired"); i_subrunasowner = PQfnumber(res, "subrunasowner"); i_subfailover = PQfnumber(res, "subfailover"); + i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo"); i_subconninfo = PQfnumber(res, "subconninfo"); i_subslotname = PQfnumber(res, "subslotname"); i_subsynccommit = PQfnumber(res, "subsynccommit"); @@ -5102,6 +5111,8 @@ getSubscriptions(Archive *fout) (strcmp(PQgetvalue(res, i, i_subrunasowner), "t") == 0); subinfo[i].subfailover = (strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0); + subinfo[i].subretainconflictinfo = + (strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0); subinfo[i].subconninfo = pg_strdup(PQgetvalue(res, i, i_subconninfo)); if (PQgetisnull(res, i, i_subslotname)) @@ -5360,6 +5371,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo) if (subinfo->subfailover) appendPQExpBufferStr(query, ", failover = true"); + if (subinfo->subretainconflictinfo) + appendPQExpBufferStr(query, ", retain_conflict_info = true"); + if (strcmp(subinfo->subsynccommit, "off") != 0) appendPQExpBuffer(query, ", synchronous_commit = %s", fmtId(subinfo->subsynccommit)); diff --git a/src/bin/pg_dump/pg_dump.h b/src/bin/pg_dump/pg_dump.h index 7417eab6aefa..945b3fce670f 100644 --- a/src/bin/pg_dump/pg_dump.h +++ b/src/bin/pg_dump/pg_dump.h @@ -708,6 +708,7 @@ typedef struct _SubscriptionInfo bool subpasswordrequired; bool subrunasowner; bool subfailover; + bool subretainconflictinfo; char *subconninfo; char *subslotname; char *subsynccommit; diff --git a/src/bin/pg_upgrade/check.c b/src/bin/pg_upgrade/check.c index 18c2d652bb6d..611bc14c1a2d 100644 --- a/src/bin/pg_upgrade/check.c +++ b/src/bin/pg_upgrade/check.c @@ -629,7 +629,7 @@ check_and_dump_old_cluster(void) * Before that the logical slots are not upgraded, so we will not be * able to upgrade the logical replication clusters completely. */ - get_subscription_count(&old_cluster); + get_subscription_info(&old_cluster); check_old_cluster_subscription_state(); } @@ -2014,9 +2014,10 @@ check_new_cluster_logical_replication_slots(void) /* * check_new_cluster_subscription_configuration() * - * Verify that the max_active_replication_origins configuration specified is - * enough for creating the subscriptions. This is required to create the - * replication origin for each subscription. + * Verify that the max_active_replication_origins and max_replication_slots + * configurations specified are enough for creating the subscriptions. This is + * required to create the replication origin and the conflict detection slot + * for each subscription. */ static void check_new_cluster_subscription_configuration(void) @@ -2024,6 +2025,8 @@ check_new_cluster_subscription_configuration(void) PGresult *res; PGconn *conn; int max_active_replication_origins; + int max_replication_slots; + int nslots_on_old; /* Subscriptions and their dependencies can be migrated since PG17. */ if (GET_MAJOR_VERSION(old_cluster.major_version) < 1700) @@ -2049,6 +2052,31 @@ check_new_cluster_subscription_configuration(void) "subscriptions (%d) on the old cluster", max_active_replication_origins, old_cluster.nsubs); + PQclear(res); + + /* Return if no subscriptions enabled the retain_conflict_info option. */ + if (!old_cluster.sub_retain_conflict_info) + { + PQfinish(conn); + check_ok(); + return; + } + + res = executeQueryOrDie(conn, "SELECT setting FROM pg_settings " + "WHERE name = 'max_replication_slots';"); + + if (PQntuples(res) != 1) + pg_fatal("could not determine parameter settings on new cluster"); + + nslots_on_old = count_old_cluster_logical_slots(); + + max_replication_slots = atoi(PQgetvalue(res, 0, 0)); + if (nslots_on_old + 1 > max_replication_slots) + pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of " + "logical replication slots on the old cluster plus one additional slot required " + "for retaining conflict detection information (%d)", + max_replication_slots, nslots_on_old + 1); + PQclear(res); PQfinish(conn); @@ -2111,6 +2139,22 @@ check_old_cluster_for_valid_slots(void) "The slot \"%s\" has not consumed the WAL yet\n", slot->slotname); } + + /* + * The name "pg_conflict_detection" (defined as + * CONFLICT_DETECTION_SLOT) has been reserved for logical + * replication conflict detection since PG18. + */ + if (strcmp(slot->slotname, "pg_conflict_detection") == 0) + { + if (script == NULL && + (script = fopen_priv(output_path, "w")) == NULL) + pg_fatal("could not open file \"%s\": %m", output_path); + + fprintf(script, + "The slot name \"%s\" is reserved\n", + slot->slotname); + } } } diff --git a/src/bin/pg_upgrade/info.c b/src/bin/pg_upgrade/info.c index 4b7a56f5b3be..69658595e0bd 100644 --- a/src/bin/pg_upgrade/info.c +++ b/src/bin/pg_upgrade/info.c @@ -752,20 +752,33 @@ count_old_cluster_logical_slots(void) } /* - * get_subscription_count() + * get_subscription_info() * - * Gets the number of subscriptions in the cluster. + * Gets the information of subscriptions in the cluster. */ void -get_subscription_count(ClusterInfo *cluster) +get_subscription_info(ClusterInfo *cluster) { PGconn *conn; PGresult *res; + int i_nsub; + int i_retain_conflict_info; conn = connectToServer(cluster, "template1"); - res = executeQueryOrDie(conn, "SELECT count(*) " - "FROM pg_catalog.pg_subscription"); - cluster->nsubs = atoi(PQgetvalue(res, 0, 0)); + if (GET_MAJOR_VERSION(cluster->major_version) >= 1800) + res = executeQueryOrDie(conn, "SELECT count(*) AS nsub," + "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) AS retain_conflict_info " + "FROM pg_catalog.pg_subscription"); + else + res = executeQueryOrDie(conn, "SELECT count(*) AS nsub," + "'f' AS retain_conflict_info " + "FROM pg_catalog.pg_subscription"); + + i_nsub = PQfnumber(res, "nsub"); + i_retain_conflict_info = PQfnumber(res, "retain_conflict_info"); + + cluster->nsubs = atoi(PQgetvalue(res, 0, i_nsub)); + cluster->sub_retain_conflict_info = (strcmp(PQgetvalue(res, 0, i_retain_conflict_info), "1") == 0); PQclear(res); PQfinish(conn); diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h index 69c965bb7d09..352c8b6f376b 100644 --- a/src/bin/pg_upgrade/pg_upgrade.h +++ b/src/bin/pg_upgrade/pg_upgrade.h @@ -302,6 +302,8 @@ typedef struct uint32 bin_version; /* version returned from pg_ctl */ const char *tablespace_suffix; /* directory specification */ int nsubs; /* number of subscriptions */ + bool sub_retain_conflict_info; /* whether a subscription enables + * retain_conflict_info. */ } ClusterInfo; @@ -441,7 +443,7 @@ FileNameMap *gen_db_file_maps(DbInfo *old_db, const char *new_pgdata); void get_db_rel_and_slot_infos(ClusterInfo *cluster); int count_old_cluster_logical_slots(void); -void get_subscription_count(ClusterInfo *cluster); +void get_subscription_info(ClusterInfo *cluster); /* option.c */ diff --git a/src/bin/pg_upgrade/t/004_subscription.pl b/src/bin/pg_upgrade/t/004_subscription.pl index c545abf65816..dc6deed5557e 100644 --- a/src/bin/pg_upgrade/t/004_subscription.pl +++ b/src/bin/pg_upgrade/t/004_subscription.pl @@ -87,6 +87,54 @@ $old_sub->start; $old_sub->safe_psql('postgres', "DROP SUBSCRIPTION regress_sub1;"); + +# ------------------------------------------------------ +# Check that pg_upgrade fails when max_replication_slots configured in the new +# cluster is less than the number of logical slots in the old cluster + 1 when +# subscription's retain_conflict_info option is enabled. +# ------------------------------------------------------ +# It is sufficient to use disabled subscription to test upgrade failure. +$publisher->safe_psql('postgres', "CREATE PUBLICATION regress_pub1"); +$old_sub->safe_psql('postgres', + "CREATE SUBSCRIPTION regress_sub1 CONNECTION '$connstr' PUBLICATION regress_pub1 WITH (enabled = false, retain_conflict_info = true)" +); + +$old_sub->stop; + +$new_sub->append_conf('postgresql.conf', "max_replication_slots = 0"); + +# pg_upgrade will fail because the new cluster has insufficient +# max_replication_slots. +command_checks_all( + [ + 'pg_upgrade', + '--no-sync', + '--old-datadir' => $old_sub->data_dir, + '--new-datadir' => $new_sub->data_dir, + '--old-bindir' => $oldbindir, + '--new-bindir' => $newbindir, + '--socketdir' => $new_sub->host, + '--old-port' => $old_sub->port, + '--new-port' => $new_sub->port, + $mode, + '--check', + ], + 1, + [ + qr/"max_replication_slots" \(0\) must be greater than or equal to the number of logical replication slots on the old cluster plus one additional slot required for retaining conflict detection information \(1\)/ + ], + [qr//], + 'run of pg_upgrade where the new cluster has insufficient max_replication_slots' +); + +# Reset max_replication_slots +$new_sub->append_conf('postgresql.conf', "max_replication_slots = 10"); + +# Cleanup +$publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1"); +$old_sub->start; +$old_sub->safe_psql('postgres', "DROP SUBSCRIPTION regress_sub1;"); + # ------------------------------------------------------ # Check that pg_upgrade refuses to run if: # a) there's a subscription with tables in a state other than 'r' (ready) or diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c index 1d08268393e3..43a6682a1315 100644 --- a/src/bin/psql/describe.c +++ b/src/bin/psql/describe.c @@ -6745,7 +6745,7 @@ describeSubscriptions(const char *pattern, bool verbose) printQueryOpt myopt = pset.popt; static const bool translate_columns[] = {false, false, false, false, false, false, false, false, false, false, false, false, false, false, - false}; + false, false}; if (pset.sversion < 100000) { @@ -6813,6 +6813,10 @@ describeSubscriptions(const char *pattern, bool verbose) appendPQExpBuffer(&buf, ", subfailover AS \"%s\"\n", gettext_noop("Failover")); + if (pset.sversion >= 180000) + appendPQExpBuffer(&buf, + ", subretainconflictinfo AS \"%s\"\n", + gettext_noop("Retain conflict info")); appendPQExpBuffer(&buf, ", subsynccommit AS \"%s\"\n" diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c index c916b9299a80..692a7d311cdc 100644 --- a/src/bin/psql/tab-complete.in.c +++ b/src/bin/psql/tab-complete.in.c @@ -2298,8 +2298,9 @@ match_previous_words(int pattern_id, /* ALTER SUBSCRIPTION SET ( */ else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "(")) COMPLETE_WITH("binary", "disable_on_error", "failover", "origin", - "password_required", "run_as_owner", "slot_name", - "streaming", "synchronous_commit", "two_phase"); + "password_required", "retain_conflict_info", + "run_as_owner", "slot_name", "streaming", + "synchronous_commit", "two_phase"); /* ALTER SUBSCRIPTION SKIP ( */ else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "(")) COMPLETE_WITH("lsn"); @@ -3722,8 +3723,9 @@ match_previous_words(int pattern_id, else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "(")) COMPLETE_WITH("binary", "connect", "copy_data", "create_slot", "disable_on_error", "enabled", "failover", "origin", - "password_required", "run_as_owner", "slot_name", - "streaming", "synchronous_commit", "two_phase"); + "password_required", "retain_conflict_info", + "run_as_owner", "slot_name", "streaming", + "synchronous_commit", "two_phase"); /* CREATE TRIGGER --- is allowed inside CREATE SCHEMA, so use TailMatches */ diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h index 20fc329992dc..0ac7c0b120cc 100644 --- a/src/include/catalog/pg_subscription.h +++ b/src/include/catalog/pg_subscription.h @@ -78,6 +78,9 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW * slots) in the upstream database are enabled * to be synchronized to the standbys. */ + bool subretainconflictinfo; /* True if information useful for + * conflict detection is retained */ + #ifdef CATALOG_VARLEN /* variable-length fields start here */ /* Connection string to the publisher */ text subconninfo BKI_FORCE_NOT_NULL; @@ -131,6 +134,8 @@ typedef struct Subscription * (i.e. the main slot and the table sync * slots) in the upstream database are enabled * to be synchronized to the standbys. */ + bool retainconflictinfo; /* True if information useful for conflict + * detection is retained */ char *conninfo; /* Connection string to the publisher */ char *slotname; /* Name of the replication slot */ char *synccommit; /* Synchronous commit setting for worker */ diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out index 1443e1d92929..bff4cc051db2 100644 --- a/src/test/regress/expected/subscription.out +++ b/src/test/regress/expected/subscription.out @@ -116,18 +116,18 @@ CREATE SUBSCRIPTION regress_testsub4 CONNECTION 'dbname=regress_doesnotexist' PU WARNING: subscription was created, but is not connected HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. \dRs+ regress_testsub4 - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN -------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub4 SET (origin = any); \dRs+ regress_testsub4 - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN -------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) DROP SUBSCRIPTION regress_testsub3; @@ -145,10 +145,10 @@ ALTER SUBSCRIPTION regress_testsub CONNECTION 'foobar'; ERROR: invalid connection string syntax: missing "=" after "foobar" in connection info string \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET PUBLICATION testpub2, testpub3 WITH (refresh = false); @@ -157,10 +157,10 @@ ALTER SUBSCRIPTION regress_testsub SET (slot_name = 'newname'); ALTER SUBSCRIPTION regress_testsub SET (password_required = false); ALTER SUBSCRIPTION regress_testsub SET (run_as_owner = true); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | off | dbname=regress_doesnotexist2 | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+------------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | off | dbname=regress_doesnotexist2 | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (password_required = true); @@ -176,10 +176,10 @@ ERROR: unrecognized subscription parameter: "create_slot" -- ok ALTER SUBSCRIPTION regress_testsub SKIP (lsn = '0/12345'); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist2 | 0/12345 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+------------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist2 | 0/12345 (1 row) -- ok - with lsn = NONE @@ -188,10 +188,10 @@ ALTER SUBSCRIPTION regress_testsub SKIP (lsn = NONE); ALTER SUBSCRIPTION regress_testsub SKIP (lsn = '0/0'); ERROR: invalid WAL location (LSN): 0/0 \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist2 | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+------------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist2 | 0/0 (1 row) BEGIN; @@ -223,10 +223,10 @@ ALTER SUBSCRIPTION regress_testsub_foo SET (synchronous_commit = foobar); ERROR: invalid value for parameter "synchronous_commit": "foobar" HINT: Available values: local, remote_write, remote_apply, on, off. \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ----------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------------+---------- - regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | local | dbname=regress_doesnotexist2 | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+------------------------------+---------- + regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | local | dbname=regress_doesnotexist2 | 0/0 (1 row) -- rename back to keep the rest simple @@ -255,19 +255,19 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB WARNING: subscription was created, but is not connected HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (binary = false); ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) DROP SUBSCRIPTION regress_testsub; @@ -279,27 +279,27 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB WARNING: subscription was created, but is not connected HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (streaming = parallel); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (streaming = false); ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) -- fail - publication already exists @@ -314,10 +314,10 @@ ALTER SUBSCRIPTION regress_testsub ADD PUBLICATION testpub1, testpub2 WITH (refr ALTER SUBSCRIPTION regress_testsub ADD PUBLICATION testpub1, testpub2 WITH (refresh = false); ERROR: publication "testpub1" is already in subscription "regress_testsub" \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) -- fail - publication used more than once @@ -332,10 +332,10 @@ ERROR: publication "testpub3" is not in subscription "regress_testsub" -- ok - delete publications ALTER SUBSCRIPTION regress_testsub DROP PUBLICATION testpub1, testpub2 WITH (refresh = false); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) DROP SUBSCRIPTION regress_testsub; @@ -371,19 +371,19 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB WARNING: subscription was created, but is not connected HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) -- we can alter streaming when two_phase enabled ALTER SUBSCRIPTION regress_testsub SET (streaming = true); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); @@ -393,10 +393,10 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB WARNING: subscription was created, but is not connected HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); @@ -409,18 +409,44 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB WARNING: subscription was created, but is not connected HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true); \dRs+ - List of subscriptions - Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Synchronous commit | Conninfo | Skip LSN ------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+-----------------------------+---------- - regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | off | dbname=regress_doesnotexist | 0/0 + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 +(1 row) + +ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); +DROP SUBSCRIPTION regress_testsub; +-- fail - retain_conflict_info must be boolean +CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = foo); +ERROR: retain_conflict_info requires a Boolean value +-- ok - but a warning will occur because track_commit_timestamp is not enabled +CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = true); +WARNING: information for detecting conflicts cannot be fully retained when "track_commit_timestamp" is disabled +WARNING: subscription was created, but is not connected +HINT: To initiate replication, you must manually create the replication slot, enable the subscription, and refresh the subscription. +\dRs+ + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | off | dbname=regress_doesnotexist | 0/0 +(1 row) + +-- ok +ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false); +\dRs+ + List of subscriptions + Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain conflict info | Synchronous commit | Conninfo | Skip LSN +-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+--------------------+-----------------------------+---------- + regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | off | dbname=regress_doesnotexist | 0/0 (1 row) ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); diff --git a/src/test/regress/sql/subscription.sql b/src/test/regress/sql/subscription.sql index 007c9e703746..c65397e5ac68 100644 --- a/src/test/regress/sql/subscription.sql +++ b/src/test/regress/sql/subscription.sql @@ -287,6 +287,22 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true); ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); DROP SUBSCRIPTION regress_testsub; +-- fail - retain_conflict_info must be boolean +CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = foo); + +-- ok - but a warning will occur because track_commit_timestamp is not enabled +CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = true); + +\dRs+ + +-- ok +ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false); + +\dRs+ + +ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); +DROP SUBSCRIPTION regress_testsub; + -- let's do some tests with pg_create_subscription rather than superuser SET SESSION AUTHORIZATION regress_subscription_user3; From d0081c7b8a7e8f5c573d4d54419b2499e040af4d Mon Sep 17 00:00:00 2001 From: Zhijie Hou Date: Thu, 24 Apr 2025 20:30:16 +0800 Subject: [PATCH 4/7] Introduce a new GUC 'max_conflict_retention_duration' This commit introduces a GUC option max_conflict_retention_duration, designed to prevent excessive accumulation of dead tuples when subscription with retain_conflict_info enabled is present and the the apply worker cannot catch up with the publisher's workload. If the time spent advancing non-removable transaction ID surpasses the max_conflict_retention_duration threshold, the apply worker would stop retaining information for conflict detection. The replication slot pg_conflict_detection will be invalidated if all apply workers associated with the subscription, where retain_conflict_info is enabled, confirm that the retention duration exceeded the max_conflict_retention_duration. In this patch, a replication slot will not be automatically re-created if it becomes invalidated. Users can disable retain_conflict_info and re-enable it after confirming that the replication slot has been dropped. An upcoming patch will include support for automatic slot recreation once at least one apply worker confirms that the retention duration is within the max_conflict_retention_duration limit. To monitor worker's conflict retention status, this patch also introduces a new column 'retain_conflict_info' in the pg_stat_subscription view. This column indicates whether the apply worker is effectively retaining conflict information. The value is set to true only if retain_conflict_info is enabled for the associated subscription, and the retention duration for conflict detection by the apply worker has not exceeded max_conflict_retention_duration. --- doc/src/sgml/config.sgml | 41 +++++ doc/src/sgml/monitoring.sgml | 13 ++ doc/src/sgml/system-views.sgml | 11 ++ src/backend/catalog/system_views.sql | 3 +- src/backend/replication/logical/launcher.c | 80 ++++++++- src/backend/replication/logical/worker.c | 166 ++++++++++++++++-- src/backend/replication/slot.c | 13 ++ src/backend/utils/misc/guc_tables.c | 13 ++ src/backend/utils/misc/postgresql.conf.sample | 1 + src/include/catalog/pg_proc.dat | 6 +- src/include/replication/logicallauncher.h | 1 + src/include/replication/slot.h | 4 +- src/include/replication/worker_internal.h | 6 + src/test/regress/expected/rules.out | 5 +- 14 files changed, 337 insertions(+), 26 deletions(-) diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml index 54afce86d766..8f2e33ec9e3a 100644 --- a/doc/src/sgml/config.sgml +++ b/doc/src/sgml/config.sgml @@ -5379,6 +5379,47 @@ ANY num_sync ( + max_conflict_retention_duration (integer) + + max_conflict_retention_duration configuration parameter + + + + + Maximum duration (in milliseconds) for which conflict + information can be retained for conflict detection by the apply worker. + The default value is 0, indicating that conflict + information is retained until it is no longer needed for detection + purposes. + + + The replication slot + pg_conflict_detection that used to + retain conflict information will be invalidated if all apply workers + associated with the subscriptions, where + retain_conflict_info is enabled, confirm that the + retention duration exceeded the + max_conflict_retention_duration. If the replication + slot is invalidated, you can disable + retain_conflict_info and re-enable it after + confirming this replication slot has been dropped. + + + This option is effective only if a subscription with + retain_conflict_info enabled is present, and the + associated apply worker is active. + + + + Note that setting a non-zero value for this option could lead to + conflict information being removed prematurely, potentially missing + some conflict detections. + + + + + diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml index d768ea065c55..5a9f17beb23d 100644 --- a/doc/src/sgml/monitoring.sgml +++ b/doc/src/sgml/monitoring.sgml @@ -2109,6 +2109,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage sender; NULL for parallel apply workers + + + + retain_conflict_info boolean + + + True if retain_conflict_info + is enabled and the duration for which conflict information is + retained for conflict detection by this apply worker does not exceed + max_conflict_retention_duration; NULL for + parallel apply workers and table synchronization workers. + + diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml index b58c52ea50f5..fe552a86fada 100644 --- a/doc/src/sgml/system-views.sgml +++ b/doc/src/sgml/system-views.sgml @@ -2936,6 +2936,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx duration. + + + conflict_retention_exceeds_max_duration means that + the duration for retaining conflict information, which is used + in logical replication conflict detection, has exceeded the maximum + allowable limit. It is set only for the slot + pg_conflict_detection, which is created when + retain_conflict_info + is enabled. + + diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql index 637a312e1987..c4443249496b 100644 --- a/src/backend/catalog/system_views.sql +++ b/src/backend/catalog/system_views.sql @@ -993,7 +993,8 @@ CREATE VIEW pg_stat_subscription AS st.last_msg_send_time, st.last_msg_receipt_time, st.latest_end_lsn, - st.latest_end_time + st.latest_end_time, + st.retain_conflict_info FROM pg_subscription su LEFT JOIN pg_stat_get_subscription(NULL) st ON (st.subid = su.oid); diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c index 07d686c6d8cd..f873a8171927 100644 --- a/src/backend/replication/logical/launcher.c +++ b/src/backend/replication/logical/launcher.c @@ -23,6 +23,7 @@ #include "access/tableam.h" #include "access/xact.h" #include "catalog/pg_subscription.h" +#include "catalog/pg_subscription_d.h" #include "catalog/pg_subscription_rel.h" #include "funcapi.h" #include "lib/dshash.h" @@ -43,6 +44,7 @@ #include "utils/memutils.h" #include "utils/pg_lsn.h" #include "utils/snapmgr.h" +#include "utils/syscache.h" /* max sleep time between cycles (3min) */ #define DEFAULT_NAPTIME_PER_CYCLE 180000L @@ -51,6 +53,7 @@ int max_logical_replication_workers = 4; int max_sync_workers_per_subscription = 2; int max_parallel_apply_workers_per_subscription = 2; +int max_conflict_retention_duration = 0; LogicalRepWorker *MyLogicalRepWorker = NULL; @@ -446,6 +449,7 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype, worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid; worker->parallel_apply = is_parallel_apply_worker; worker->oldest_nonremovable_xid = InvalidFullTransactionId; + worker->stop_conflict_info_retention = false; worker->last_lsn = InvalidXLogRecPtr; TIMESTAMP_NOBEGIN(worker->last_send_time); TIMESTAMP_NOBEGIN(worker->last_recv_time); @@ -1157,7 +1161,8 @@ ApplyLauncherMain(Datum main_arg) MemoryContext oldctx; long wait_time = DEFAULT_NAPTIME_PER_CYCLE; bool can_advance_xmin = true; - bool retain_conflict_info = false; + int nretain_conflict_info = 0; + int nstop_retention = 0; FullTransactionId xmin = InvalidFullTransactionId; CHECK_FOR_INTERRUPTS(); @@ -1186,7 +1191,7 @@ ApplyLauncherMain(Datum main_arg) */ if (sub->retainconflictinfo) { - retain_conflict_info = true; + nretain_conflict_info++; can_advance_xmin &= sub->enabled; /* @@ -1212,22 +1217,32 @@ ApplyLauncherMain(Datum main_arg) * the new xmin for advancing the replication slot used in * conflict detection. */ - if (sub->retainconflictinfo && can_advance_xmin) + if (sub->retainconflictinfo) { FullTransactionId nonremovable_xid; + bool stop_conflict_info_retention; SpinLockAcquire(&w->relmutex); nonremovable_xid = w->oldest_nonremovable_xid; + stop_conflict_info_retention = w->stop_conflict_info_retention; SpinLockRelease(&w->relmutex); + /* + * Skip collecting oldest_nonremovable_xid for workers + * that have stopped conflict retention. + */ + if (stop_conflict_info_retention) + nstop_retention++; + /* * Stop advancing xmin if an invalid non-removable * transaction ID is found, otherwise update xmin. */ - if (!FullTransactionIdIsValid(nonremovable_xid)) + else if (!FullTransactionIdIsValid(nonremovable_xid)) can_advance_xmin = false; - else if (!FullTransactionIdIsValid(xmin) || - FullTransactionIdPrecedes(nonremovable_xid, xmin)) + else if (can_advance_xmin && + (!FullTransactionIdIsValid(xmin) || + FullTransactionIdPrecedes(nonremovable_xid, xmin))) xmin = nonremovable_xid; } @@ -1272,11 +1287,34 @@ ApplyLauncherMain(Datum main_arg) } } + /* + * Do nothing if the replication slot is invalidated due to conflict + * retention duration. + */ + if (nretain_conflict_info && + MyReplicationSlot->data.invalidated != RS_INVAL_NONE) + { + Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION); + } + + /* + * Invalidate the conflict slot if all workers with + * retain_conflict_info enabled have stopped further conflict + * retention. + */ + else if (nstop_retention && nretain_conflict_info == nstop_retention) + { + ReplicationSlotRelease(); + InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION, + InvalidXLogRecPtr, InvalidOid, + InvalidTransactionId); + } + /* * Maintain the xmin value of the replication slot for conflict * detection if needed. */ - if (retain_conflict_info) + else if (nretain_conflict_info) { if (can_advance_xmin) advance_conflict_slot_xmin(xmin); @@ -1470,7 +1508,7 @@ GetLeaderApplyWorkerPid(pid_t pid) Datum pg_stat_get_subscription(PG_FUNCTION_ARGS) { -#define PG_STAT_GET_SUBSCRIPTION_COLS 10 +#define PG_STAT_GET_SUBSCRIPTION_COLS 11 Oid subid = PG_ARGISNULL(0) ? InvalidOid : PG_GETARG_OID(0); int i; ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo; @@ -1547,6 +1585,32 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS) elog(ERROR, "unknown worker type"); } + /* + * Only the leader apply worker manages conflict retention (see + * maybe_advance_nonremovable_xid() for details). + */ + if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker)) + { + HeapTuple tup; + Form_pg_subscription subform; + + tup = SearchSysCache1(SUBSCRIPTIONOID, + ObjectIdGetDatum(worker.subid)); + + if (!HeapTupleIsValid(tup)) + elog(ERROR, "cache lookup failed for subscription %u", + worker.subid); + + subform = (Form_pg_subscription) GETSTRUCT(tup); + + values[10] = subform->subretainconflictinfo && + !worker.stop_conflict_info_retention; + + ReleaseSysCache(tup); + } + else + nulls[10] = true; + tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls); diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c index 471a95da3fb1..23e6fb7f2212 100644 --- a/src/backend/replication/logical/worker.c +++ b/src/backend/replication/logical/worker.c @@ -173,6 +173,7 @@ #include "replication/logicalrelation.h" #include "replication/logicalworker.h" #include "replication/origin.h" +#include "replication/slot.h" #include "replication/walreceiver.h" #include "replication/worker_internal.h" #include "rewrite/rewriteHandler.h" @@ -451,8 +452,11 @@ static void request_publisher_status(RetainConflictInfoData *data); static void wait_for_publisher_status(RetainConflictInfoData *data, bool status_received); static void wait_for_local_flush(RetainConflictInfoData *data); +static void reset_conflict_info_fields(RetainConflictInfoData *data); +static bool should_stop_conflict_info_retention(RetainConflictInfoData *data); static void adjust_xid_advance_interval(RetainConflictInfoData *data, bool new_xid_found); +static void update_conflict_retention_status(void); static void apply_handle_commit_internal(LogicalRepCommitData *commit_data); static void apply_handle_insert_internal(ApplyExecutionData *edata, @@ -3864,7 +3868,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received) * Ensure to wake up when it's possible to attempt to advance the * non-removable transaction ID. */ - if (data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval) + if (!MyLogicalRepWorker->stop_conflict_info_retention && + data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval) wait_time = Min(wait_time, data.xid_advance_interval); rc = WaitLatchOrSocket(MyLatch, @@ -4121,6 +4126,10 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data, if (!am_leader_apply_worker()) return; + /* Exit early if we have already stopped retaining */ + if (MyLogicalRepWorker->stop_conflict_info_retention) + return; + switch (data->phase) { case RCI_GET_CANDIDATE_XID: @@ -4248,6 +4257,13 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received) if (!status_received) return; + /* + * Stop retaining conflict information if required (See + * should_stop_conflict_info_retention() for details). + */ + if (should_stop_conflict_info_retention(data)) + return; + remote_next_full_xid = FullTransactionIdFromEpochAndXid(data->remote_epoch, data->remote_nextxid); @@ -4302,6 +4318,13 @@ wait_for_local_flush(RetainConflictInfoData *data) errmsg("oldest_nonremovable_xid transaction ID may be advanced prematurely"), errdetail("The clock on the publisher is behind that of the subscriber.")); + /* + * Stop retaining conflict information if required (See + * should_stop_conflict_info_retention() for details). + */ + if (should_stop_conflict_info_retention(data)) + return; + /* * Do not attempt to advance the non-removable transaction ID when table * sync is in progress. During this time, changes from a single @@ -4344,14 +4367,18 @@ wait_for_local_flush(RetainConflictInfoData *data) data->flushpos_update_time = data->last_recv_time; } - /* Return to wait for the changes to be applied */ + /* + * Return if changes up to the remote_lsn have not been applied and + * flushed. + */ if (last_flushpos < data->remote_lsn) return; /* - * Reaching here means the remote WAL position has been received, and all - * transactions up to that position on the publisher have been applied and - * flushed locally. So, we can advance the non-removable transaction ID. + * Reaching here means the remote WAL position has been received, and + * all transactions up to that position on the publisher have been + * applied and flushed locally. So, we can advance the non-removable + * transaction ID. */ SpinLockAcquire(&MyLogicalRepWorker->relmutex); MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid; @@ -4364,10 +4391,19 @@ wait_for_local_flush(RetainConflictInfoData *data) /* Notify launcher to update the xmin of the conflict slot */ ApplyLauncherWakeup(); - /* - * Reset all data fields except those used to determine the timing for the - * next round of transaction ID advancement. - */ + reset_conflict_info_fields(data); + + /* process the next phase */ + maybe_advance_nonremovable_xid(data, false); +} + +/* + * Reset all data fields of RetainConflictInfoData except those used to + * determine the timing for the next round of transaction ID advancement. + */ +static void +reset_conflict_info_fields(RetainConflictInfoData *data) +{ data->phase = RCI_GET_CANDIDATE_XID; data->remote_lsn = InvalidXLogRecPtr; data->remote_oldestxid = InvalidTransactionId; @@ -4377,9 +4413,60 @@ wait_for_local_flush(RetainConflictInfoData *data) data->candidate_xid = InvalidFullTransactionId; data->reply_time = 0; data->flushpos_update_time = 0; +} - /* process the next phase */ - maybe_advance_nonremovable_xid(data, false); +/* + * Check whether conflict information retention should be stopped because the + * wait time has exceeded the maximum limit (max_conflict_retention_duration). + * + * If retention should be stopped, set + * LogicalRepWorker->stop_conflict_info_retention to true, notify the launcher to + * invalidate the slot, and return true. Return false otherwise. + * + * Currently, the retention will not resume automatically unless user manually + * disables retain_conflict_info and re-enables it after confirming that the + * replication slot has been dropped. + */ +static bool +should_stop_conflict_info_retention(RetainConflictInfoData *data) +{ + TimestampTz now; + + if (!max_conflict_retention_duration) + return false; + + /* + * Use last_recv_time when applying changes in the loop to avoid + * unnecessary system time retrieval. If last_recv_time is not + * available, obtain the current timestamp. + */ + now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp(); + + /* + * Return if the wait time has not exceeded the maximum limit + * (max_conflict_retention_duration). + */ + if (!TimestampDifferenceExceeds(data->candidate_xid_time, now, + max_conflict_retention_duration)) + return false; + + ereport(LOG, + errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information", + MySubscription->name), + errdetail("The time spent advancing the non-removable transaction ID has exceeded the maximum limit of %u ms.", + max_conflict_retention_duration)); + + SpinLockAcquire(&MyLogicalRepWorker->relmutex); + MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId; + MyLogicalRepWorker->stop_conflict_info_retention = true; + SpinLockRelease(&MyLogicalRepWorker->relmutex); + + /* Notify launcher to invalidate the conflict slot */ + ApplyLauncherWakeup(); + + reset_conflict_info_fields(data); + + return true; } /* @@ -4423,6 +4510,51 @@ adjust_xid_advance_interval(RetainConflictInfoData *data, bool new_xid_found) } } +/* + * Update the conflict retention status for the current apply worker. It checks + * whether the worker should stop retaining conflict information due to + * invalidation of the replication slot ("pg_conflict_detection"). + * + * Currently, the replication slot is invalidated only if the duration for + * retaining conflict information exceeds the allowed maximum. + */ +static void +update_conflict_retention_status(void) +{ + ReplicationSlotInvalidationCause cause = RS_INVAL_NONE; + ReplicationSlot *slot; + + /* Exit early if retaining conflict information is not required */ + if (!MySubscription->retainconflictinfo) + return; + + /* + * Only the leader apply worker manages conflict retention (see + * maybe_advance_nonremovable_xid() for details). + */ + if (!am_leader_apply_worker()) + return; + + LWLockAcquire(ReplicationSlotControlLock, LW_SHARED); + + slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, false); + + if (slot) + { + SpinLockAcquire(&slot->mutex); + cause = slot->data.invalidated; + SpinLockRelease(&slot->mutex); + + Assert(cause == RS_INVAL_NONE || cause == RS_INVAL_CONFLICT_RETENTION_DURATION); + } + + LWLockRelease(ReplicationSlotControlLock); + + SpinLockAcquire(&MyLogicalRepWorker->relmutex); + MyLogicalRepWorker->stop_conflict_info_retention = cause != RS_INVAL_NONE; + SpinLockRelease(&MyLogicalRepWorker->relmutex); +} + /* * Exit routine for apply workers due to subscription parameter changes. */ @@ -4594,6 +4726,16 @@ maybe_reread_subscription(void) CommitTransactionCommand(); MySubscriptionValid = true; + + /* + * Update worker status to avoid unnecessary conflict retention if the + * replication slot ("pg_conflict_detection") was invalidated prior to + * enabling the retain_conflict_info option. This is also necessary to + * restart conflict retention if the user has disabled and subsequently + * re-enabled the retain_conflict_info option, resulting in the + * replication slot being recreated. + */ + update_conflict_retention_status(); } /* @@ -5240,6 +5382,8 @@ InitializeLogRepWorker(void) MySubscription->name))); CommitTransactionCommand(); + + update_conflict_retention_status(); } /* diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c index 668279cb4e8e..bce4af1bdcdf 100644 --- a/src/backend/replication/slot.c +++ b/src/backend/replication/slot.c @@ -116,6 +116,7 @@ static const SlotInvalidationCauseMap SlotInvalidationCauses[] = { {RS_INVAL_HORIZON, "rows_removed"}, {RS_INVAL_WAL_LEVEL, "wal_level_insufficient"}, {RS_INVAL_IDLE_TIMEOUT, "idle_timeout"}, + {RS_INVAL_CONFLICT_RETENTION_DURATION, "conflict_retention_exceeds_max_duration"}, }; /* @@ -1610,6 +1611,11 @@ ReportSlotInvalidation(ReplicationSlotInvalidationCause cause, "idle_replication_slot_timeout"); break; } + case RS_INVAL_CONFLICT_RETENTION_DURATION: + appendStringInfo(&err_detail, + _("The duration for retaining conflict information exceeds the configured \"%s\" limit of %d milliseconds"), + "max_conflict_retention_duration", max_conflict_retention_duration); + break; case RS_INVAL_NONE: pg_unreachable(); } @@ -1728,6 +1734,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s, } } + if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION) + { + if (IsReservedSlotName(NameStr(s->data.name))) + return RS_INVAL_CONFLICT_RETENTION_DURATION; + } + return RS_INVAL_NONE; } @@ -1995,6 +2007,7 @@ InvalidatePossiblyObsoleteSlot(uint32 possible_causes, * - RS_INVAL_WAL_LEVEL: is logical and wal_level is insufficient * - RS_INVAL_IDLE_TIMEOUT: has been idle longer than the configured * "idle_replication_slot_timeout" duration. + * - RS_INVAL_CONFLICT_RETENTION_DURATION: is "pg_conflict_detection" * * Note: This function attempts to invalidate the slot for multiple possible * causes in a single pass, minimizing redundant iterations. The "cause" diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c index 2f8cbd867599..e8249ba84915 100644 --- a/src/backend/utils/misc/guc_tables.c +++ b/src/backend/utils/misc/guc_tables.c @@ -3388,6 +3388,19 @@ struct config_int ConfigureNamesInt[] = NULL, NULL, NULL }, + { + {"max_conflict_retention_duration", + PGC_SIGHUP, + REPLICATION_SUBSCRIBERS, + gettext_noop("Maximum duration for retaining information used in logical replication conflict detection."), + NULL, + GUC_UNIT_MS + }, + &max_conflict_retention_duration, + 0, 0, INT_MAX, + NULL, NULL, NULL + }, + { {"log_rotation_age", PGC_SIGHUP, LOGGING_WHERE, gettext_noop("Sets the amount of time to wait before forcing " diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample index 34826d01380b..4cabb0e7c487 100644 --- a/src/backend/utils/misc/postgresql.conf.sample +++ b/src/backend/utils/misc/postgresql.conf.sample @@ -400,6 +400,7 @@ # (change requires restart) #max_sync_workers_per_subscription = 2 # taken from max_logical_replication_workers #max_parallel_apply_workers_per_subscription = 2 # taken from max_logical_replication_workers +#max_conflict_retention_duration = 0 # in milliseconds; 0 disables #------------------------------------------------------------------------------ diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat index 62beb71da288..2e7a337762fe 100644 --- a/src/include/catalog/pg_proc.dat +++ b/src/include/catalog/pg_proc.dat @@ -5695,9 +5695,9 @@ proname => 'pg_stat_get_subscription', prorows => '10', proisstrict => 'f', proretset => 't', provolatile => 's', proparallel => 'r', prorettype => 'record', proargtypes => 'oid', - proallargtypes => '{oid,oid,oid,int4,int4,pg_lsn,timestamptz,timestamptz,pg_lsn,timestamptz,text}', - proargmodes => '{i,o,o,o,o,o,o,o,o,o,o}', - proargnames => '{subid,subid,relid,pid,leader_pid,received_lsn,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,worker_type}', + proallargtypes => '{oid,oid,oid,int4,int4,pg_lsn,timestamptz,timestamptz,pg_lsn,timestamptz,text,bool}', + proargmodes => '{i,o,o,o,o,o,o,o,o,o,o,o}', + proargnames => '{subid,subid,relid,pid,leader_pid,received_lsn,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time,worker_type,retain_conflict_info}', prosrc => 'pg_stat_get_subscription' }, { oid => '2026', descr => 'statistics: current backend PID', proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r', diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h index 7b29f1814db3..d347d3d67f69 100644 --- a/src/include/replication/logicallauncher.h +++ b/src/include/replication/logicallauncher.h @@ -15,6 +15,7 @@ extern PGDLLIMPORT int max_logical_replication_workers; extern PGDLLIMPORT int max_sync_workers_per_subscription; extern PGDLLIMPORT int max_parallel_apply_workers_per_subscription; +extern PGDLLIMPORT int max_conflict_retention_duration; extern void ApplyLauncherRegister(void); extern void ApplyLauncherMain(Datum main_arg); diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h index e03e123a2ff9..a8d0844e9cfc 100644 --- a/src/include/replication/slot.h +++ b/src/include/replication/slot.h @@ -66,10 +66,12 @@ typedef enum ReplicationSlotInvalidationCause RS_INVAL_WAL_LEVEL = (1 << 2), /* idle slot timeout has occurred */ RS_INVAL_IDLE_TIMEOUT = (1 << 3), + /* duration of conflict info retention exceeds the maximum limit */ + RS_INVAL_CONFLICT_RETENTION_DURATION = (1 << 4), } ReplicationSlotInvalidationCause; /* Maximum number of invalidation causes */ -#define RS_INVAL_MAX_CAUSES 4 +#define RS_INVAL_MAX_CAUSES 5 /* * On-Disk data of a replication slot, preserved across restarts. diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h index b09486017f4a..0fdf49a1938d 100644 --- a/src/include/replication/worker_internal.h +++ b/src/include/replication/worker_internal.h @@ -103,6 +103,12 @@ typedef struct LogicalRepWorker */ FullTransactionId oldest_nonremovable_xid; + /* + * Indicates whether the apply worker has stopped retaining conflict + * information. This is used only when retain_conflict_info is enabled. + */ + bool stop_conflict_info_retention; + /* Stats. */ XLogRecPtr last_lsn; TimestampTz last_send_time; diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out index 6cf828ca8d0d..79ed5233edb6 100644 --- a/src/test/regress/expected/rules.out +++ b/src/test/regress/expected/rules.out @@ -2165,9 +2165,10 @@ pg_stat_subscription| SELECT su.oid AS subid, st.last_msg_send_time, st.last_msg_receipt_time, st.latest_end_lsn, - st.latest_end_time + st.latest_end_time, + st.retain_conflict_info FROM (pg_subscription su - LEFT JOIN pg_stat_get_subscription(NULL::oid) st(subid, relid, pid, leader_pid, received_lsn, last_msg_send_time, last_msg_receipt_time, latest_end_lsn, latest_end_time, worker_type) ON ((st.subid = su.oid))); + LEFT JOIN pg_stat_get_subscription(NULL::oid) st(subid, relid, pid, leader_pid, received_lsn, last_msg_send_time, last_msg_receipt_time, latest_end_lsn, latest_end_time, worker_type, retain_conflict_info) ON ((st.subid = su.oid))); pg_stat_subscription_stats| SELECT ss.subid, s.subname, ss.apply_error_count, From e7ae97b8b44ae63ec4313a14db9324217c79ce68 Mon Sep 17 00:00:00 2001 From: Zhijie Hou Date: Thu, 24 Apr 2025 19:40:33 +0800 Subject: [PATCH 5/7] Re-create the replication slot if the conflict retention duration reduced The patch allows the launcher to drop and re-create the invalidated slot, if at least one apply worker has confirmed that the retention duration is now within the max_conflict_retention_duration. --- doc/src/sgml/config.sgml | 5 +- src/backend/replication/logical/launcher.c | 37 ++++--- src/backend/replication/logical/worker.c | 114 +++++++-------------- 3 files changed, 61 insertions(+), 95 deletions(-) diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml index 8f2e33ec9e3a..12f9d3dd9731 100644 --- a/doc/src/sgml/config.sgml +++ b/doc/src/sgml/config.sgml @@ -5403,7 +5403,10 @@ ANY num_sync ( name), + errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.", + LSN_FORMAT_ARGS(data->remote_lsn), + max_conflict_retention_duration)); + /* * Reaching here means the remote WAL position has been received, and * all transactions up to that position on the publisher have been @@ -4382,6 +4390,7 @@ wait_for_local_flush(RetainConflictInfoData *data) */ SpinLockAcquire(&MyLogicalRepWorker->relmutex); MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid; + MyLogicalRepWorker->stop_conflict_info_retention = false; SpinLockRelease(&MyLogicalRepWorker->relmutex); elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u", @@ -4423,9 +4432,8 @@ reset_conflict_info_fields(RetainConflictInfoData *data) * LogicalRepWorker->stop_conflict_info_retention to true, notify the launcher to * invalidate the slot, and return true. Return false otherwise. * - * Currently, the retention will not resume automatically unless user manually - * disables retain_conflict_info and re-enables it after confirming that the - * replication slot has been dropped. + * The retention will resume automatically if the worker has confirmed that the + * retention duration is now within the max_conflict_retention_duration. */ static bool should_stop_conflict_info_retention(RetainConflictInfoData *data) @@ -4450,19 +4458,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *data) max_conflict_retention_duration)) return false; - ereport(LOG, - errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information", - MySubscription->name), - errdetail("The time spent advancing the non-removable transaction ID has exceeded the maximum limit of %u ms.", - max_conflict_retention_duration)); - - SpinLockAcquire(&MyLogicalRepWorker->relmutex); - MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId; - MyLogicalRepWorker->stop_conflict_info_retention = true; - SpinLockRelease(&MyLogicalRepWorker->relmutex); - - /* Notify launcher to invalidate the conflict slot */ - ApplyLauncherWakeup(); + /* + * Log a message and reset relevant data when the worker is about to stop + * retaining conflict information. + */ + if (!MyLogicalRepWorker->stop_conflict_info_retention) + { + ereport(LOG, + errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information", + MySubscription->name), + errdetail("The time spent advancing the non-removable transaction ID has exceeded the maximum limit of %u ms.", + max_conflict_retention_duration)); + + SpinLockAcquire(&MyLogicalRepWorker->relmutex); + MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId; + MyLogicalRepWorker->stop_conflict_info_retention = true; + SpinLockRelease(&MyLogicalRepWorker->relmutex); + + /* Notify launcher to invalidate the conflict slot */ + ApplyLauncherWakeup(); + } reset_conflict_info_fields(data); @@ -4510,51 +4525,6 @@ adjust_xid_advance_interval(RetainConflictInfoData *data, bool new_xid_found) } } -/* - * Update the conflict retention status for the current apply worker. It checks - * whether the worker should stop retaining conflict information due to - * invalidation of the replication slot ("pg_conflict_detection"). - * - * Currently, the replication slot is invalidated only if the duration for - * retaining conflict information exceeds the allowed maximum. - */ -static void -update_conflict_retention_status(void) -{ - ReplicationSlotInvalidationCause cause = RS_INVAL_NONE; - ReplicationSlot *slot; - - /* Exit early if retaining conflict information is not required */ - if (!MySubscription->retainconflictinfo) - return; - - /* - * Only the leader apply worker manages conflict retention (see - * maybe_advance_nonremovable_xid() for details). - */ - if (!am_leader_apply_worker()) - return; - - LWLockAcquire(ReplicationSlotControlLock, LW_SHARED); - - slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, false); - - if (slot) - { - SpinLockAcquire(&slot->mutex); - cause = slot->data.invalidated; - SpinLockRelease(&slot->mutex); - - Assert(cause == RS_INVAL_NONE || cause == RS_INVAL_CONFLICT_RETENTION_DURATION); - } - - LWLockRelease(ReplicationSlotControlLock); - - SpinLockAcquire(&MyLogicalRepWorker->relmutex); - MyLogicalRepWorker->stop_conflict_info_retention = cause != RS_INVAL_NONE; - SpinLockRelease(&MyLogicalRepWorker->relmutex); -} - /* * Exit routine for apply workers due to subscription parameter changes. */ @@ -4726,16 +4696,6 @@ maybe_reread_subscription(void) CommitTransactionCommand(); MySubscriptionValid = true; - - /* - * Update worker status to avoid unnecessary conflict retention if the - * replication slot ("pg_conflict_detection") was invalidated prior to - * enabling the retain_conflict_info option. This is also necessary to - * restart conflict retention if the user has disabled and subsequently - * re-enabled the retain_conflict_info option, resulting in the - * replication slot being recreated. - */ - update_conflict_retention_status(); } /* @@ -5382,8 +5342,6 @@ InitializeLogRepWorker(void) MySubscription->name))); CommitTransactionCommand(); - - update_conflict_retention_status(); } /* From 6c67510786d35d4c35bea74a5cb710e8ec72fefd Mon Sep 17 00:00:00 2001 From: Zhijie Hou Date: Thu, 24 Apr 2025 11:21:48 +0800 Subject: [PATCH 6/7] Add a tap test to verify the management of the new replication slot --- src/test/subscription/t/035_conflicts.pl | 152 ++++++++++++++++++++++- 1 file changed, 150 insertions(+), 2 deletions(-) diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl index 2a7a8239a296..3bfe1631934e 100644 --- a/src/test/subscription/t/035_conflicts.pl +++ b/src/test/subscription/t/035_conflicts.pl @@ -1,6 +1,6 @@ # Copyright (c) 2025, PostgreSQL Global Development Group -# Test the conflict detection of conflict type 'multiple_unique_conflicts'. +# Test conflicts in logical replication use strict; use warnings FATAL => 'all'; use PostgreSQL::Test::Cluster; @@ -18,7 +18,7 @@ # Create a subscriber node my $node_subscriber = PostgreSQL::Test::Cluster->new('subscriber'); -$node_subscriber->init; +$node_subscriber->init(allows_streaming => 'logical'); $node_subscriber->start; # Create a table on publisher @@ -145,4 +145,152 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert'); +############################################################################### +# Setup a bidirectional logical replication between node_A & node_B +############################################################################### + +# Initialize nodes. + +# node_A. Increase the log_min_messages setting to DEBUG2 to debug test +# failures. Disable autovacuum to avoid generating xid that could affect the +# replication slot's xmin value. +my $node_A = $node_publisher; +$node_A->append_conf( + 'postgresql.conf', + qq{autovacuum = off + log_min_messages = 'debug2'}); +$node_A->restart; + +# node_B +my $node_B = $node_subscriber; +$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on"); +$node_B->restart; + +# Create table on node_A +$node_A->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY, b int)"); + +# Create the same table on node_B +$node_B->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY, b int)"); + +my $subname_AB = 'tap_sub_a_b'; +my $subname_BA = 'tap_sub_b_a'; + +# Setup logical replication +# node_A (pub) -> node_B (sub) +my $node_A_connstr = $node_A->connstr . ' dbname=postgres'; +$node_A->safe_psql('postgres', "CREATE PUBLICATION tap_pub_A FOR TABLE tab"); +$node_B->safe_psql( + 'postgres', " + CREATE SUBSCRIPTION $subname_BA + CONNECTION '$node_A_connstr application_name=$subname_BA' + PUBLICATION tap_pub_A + WITH (origin = none, retain_conflict_info = true)"); + +# node_B (pub) -> node_A (sub) +my $node_B_connstr = $node_B->connstr . ' dbname=postgres'; +$node_B->safe_psql('postgres', "CREATE PUBLICATION tap_pub_B FOR TABLE tab"); +$node_A->safe_psql( + 'postgres', " + CREATE SUBSCRIPTION $subname_AB + CONNECTION '$node_B_connstr application_name=$subname_AB' + PUBLICATION tap_pub_B + WITH (origin = none, copy_data = off, retain_conflict_info = true)"); + +# Wait for initial table sync to finish +$node_A->wait_for_subscription_sync($node_B, $subname_AB); +$node_B->wait_for_subscription_sync($node_A, $subname_BA); + +is(1, 1, 'Bidirectional replication setup is complete'); + +# Confirm that the additional replication slot is created on both nodes and the +# xmin value is valid. + +ok( $node_A->poll_query_until( + 'postgres', + "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'" + ), + "the xmin value of slot 'pg_conflict_detection' is valid on Node A"); + +ok( $node_B->poll_query_until( + 'postgres', + "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'" + ), + "the xmin value of slot 'pg_conflict_detection' is valid on Node B"); + +my $result = $node_A->safe_psql('postgres', + "SELECT retain_conflict_info FROM pg_stat_subscription WHERE subname='$subname_AB';"); +is($result, qq(t), 'worker on node A retains conflict information'); +$result = $node_B->safe_psql('postgres', + "SELECT retain_conflict_info FROM pg_stat_subscription WHERE subname='$subname_BA';"); +is($result, qq(t), 'worker on node B retains conflict information'); + +############################################################################### +# Check that dead tuples on node A cannot be cleaned by VACUUM until the +# concurrent transactions on Node B have been applied and flushed on Node A. +############################################################################### + +# Insert a record +$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);"); +$node_A->wait_for_catchup($subname_BA); + +$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;"); +is($result, qq(1|1 +2|2), 'check replicated insert on node B'); + +# Disable the logical replication from node B to node A +$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE"); + +my $log_location = -s $node_B->logfile; + +$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;"); +$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;"); + +$node_A->wait_for_catchup($subname_BA); + +my ($cmdret, $stdout, $stderr) = $node_A->psql( + 'postgres', qq(VACUUM (verbose) public.tab;) +); + +ok( $stderr =~ + qr/1 are dead but not yet removable/, + 'the deleted column is non-removable'); + +$node_A->safe_psql( + 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;"); +$node_B->wait_for_catchup($subname_AB); + +# Remember the next transaction ID to be assigned +my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;"); + +# Confirm that the xmin value is updated +ok( $node_A->poll_query_until( + 'postgres', + "SELECT xmin = $next_xid from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'" + ), + "the xmin value of slot 'pg_conflict_detection' is updated on Node A"); + +############################################################################### +# Check that the replication slot pg_conflict_detection is dropped after +# removing all the subscriptions. +############################################################################### + +$node_B->safe_psql( + 'postgres', "DROP SUBSCRIPTION $subname_BA"); + +ok( $node_B->poll_query_until( + 'postgres', + "SELECT count(*) = 0 FROM pg_replication_slots WHERE slot_name = 'pg_conflict_detection'" + ), + "the slot 'pg_conflict_detection' has been dropped on Node B"); + +$node_A->safe_psql( + 'postgres', "DROP SUBSCRIPTION $subname_AB"); + +ok( $node_A->poll_query_until( + 'postgres', + "SELECT count(*) = 0 FROM pg_replication_slots WHERE slot_name = 'pg_conflict_detection'" + ), + "the slot 'pg_conflict_detection' has been dropped on Node A"); + + done_testing(); From 8131dd030476e4a82dea51bcb80dd4257e12a9ee Mon Sep 17 00:00:00 2001 From: Zhijie Hou Date: Thu, 24 Apr 2025 17:33:10 +0800 Subject: [PATCH 7/7] Support the conflict detection for update_deleted This patch supports detecting update_deleted conflicts during update operations. If the target row cannot be found when applying update operations, we perform an additional scan of the table using snapshotAny. This scan aims to locate the most recently deleted row that matches the old column values from the remote update operation and has not yet been removed by VACUUM. If any such tuples are found, we report the update_deleted conflict along with the origin and transaction information that deleted the tuple. --- doc/src/sgml/logical-replication.sgml | 15 +++ doc/src/sgml/monitoring.sgml | 11 ++ src/backend/catalog/system_views.sql | 1 + src/backend/executor/execReplication.c | 138 ++++++++++++++++++++- src/backend/replication/logical/conflict.c | 22 ++++ src/backend/replication/logical/worker.c | 38 ++++-- src/backend/utils/adt/pgstatfuncs.c | 18 +-- src/include/catalog/pg_proc.dat | 6 +- src/include/executor/executor.h | 7 +- src/include/replication/conflict.h | 3 + src/test/regress/expected/rules.out | 3 +- src/test/subscription/t/035_conflicts.pl | 25 +++- 12 files changed, 260 insertions(+), 27 deletions(-) diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml index 9fcb7c0ff73b..9342272fd17e 100644 --- a/doc/src/sgml/logical-replication.sgml +++ b/doc/src/sgml/logical-replication.sgml @@ -1818,6 +1818,21 @@ test_sub=# SELECT * from tab_gen_to_gen; + + update_deleted + + + The tuple to be updated was deleted by another origin. The update will + simply be skipped in this scenario. + Note that this conflict can only be detected when + + and retain_conflict_info + are enabled. Note that if a tuple cannot be found due to the table being + truncated only a update_missing conflict will arise. + arise + + + update_origin_differs diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml index 5a9f17beb23d..81b202bbebf0 100644 --- a/doc/src/sgml/monitoring.sgml +++ b/doc/src/sgml/monitoring.sgml @@ -2207,6 +2207,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage + + + confl_update_deleted bigint + + + Number of times the tuple to be updated was deleted by another origin + during the application of changes. See + for details about this conflict. + + + confl_update_origin_differs bigint diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql index c4443249496b..d90289f57da4 100644 --- a/src/backend/catalog/system_views.sql +++ b/src/backend/catalog/system_views.sql @@ -1395,6 +1395,7 @@ CREATE VIEW pg_stat_subscription_stats AS ss.apply_error_count, ss.sync_error_count, ss.confl_insert_exists, + ss.confl_update_deleted, ss.confl_update_origin_differs, ss.confl_update_exists, ss.confl_update_missing, diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c index 53ddd25c42db..ce8eec5ca20c 100644 --- a/src/backend/executor/execReplication.c +++ b/src/backend/executor/execReplication.c @@ -14,12 +14,14 @@ #include "postgres.h" +#include "access/commit_ts.h" #include "access/genam.h" #include "access/gist.h" #include "access/relscan.h" #include "access/tableam.h" #include "access/transam.h" #include "access/xact.h" +#include "access/heapam.h" #include "catalog/pg_am_d.h" #include "commands/trigger.h" #include "executor/executor.h" @@ -27,6 +29,7 @@ #include "replication/conflict.h" #include "replication/logicalrelation.h" #include "storage/lmgr.h" +#include "storage/procarray.h" #include "utils/builtins.h" #include "utils/lsyscache.h" #include "utils/rel.h" @@ -36,7 +39,7 @@ static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2, - TypeCacheEntry **eq); + TypeCacheEntry **eq, Bitmapset *columns); /* * Setup a ScanKey for a search in the relation 'rel' for a tuple 'key' that @@ -221,7 +224,7 @@ RelationFindReplTupleByIndex(Relation rel, Oid idxoid, if (eq == NULL) eq = palloc0(sizeof(*eq) * outslot->tts_tupleDescriptor->natts); - if (!tuples_equal(outslot, searchslot, eq)) + if (!tuples_equal(outslot, searchslot, eq, NULL)) continue; } @@ -277,10 +280,13 @@ RelationFindReplTupleByIndex(Relation rel, Oid idxoid, /* * Compare the tuples in the slots by checking if they have equal values. + * + * If 'columns' is not null, only the columns specified within it will be + * considered for the equality check, ignoring all other columns. */ static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2, - TypeCacheEntry **eq) + TypeCacheEntry **eq, Bitmapset *columns) { int attrnum; @@ -305,6 +311,14 @@ tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2, if (att->attisdropped || att->attgenerated) continue; + /* + * Ignore columns that are not listed for checking. + */ + if (columns && + !bms_is_member(att->attnum - FirstLowInvalidHeapAttributeNumber, + columns)) + continue; + /* * If one value is NULL and other is not, then they are certainly not * equal @@ -380,7 +394,7 @@ RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode, /* Try to find the tuple */ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot)) { - if (!tuples_equal(scanslot, searchslot, eq)) + if (!tuples_equal(scanslot, searchslot, eq, NULL)) continue; found = true; @@ -455,6 +469,122 @@ BuildConflictIndexInfo(ResultRelInfo *resultRelInfo, Oid conflictindex) } } +/* + * Searches the relation 'rel' for the most recently deleted tuple that matches + * the values in 'searchslot' and is not yet removable by VACUUM. The function + * returns the transaction ID, origin, and commit timestamp of the transaction + * that deleted this tuple. + * + * The commit timestamp of the transaction that deleted the tuple is used to + * determine whether the tuple is the most recently deleted one. + * + * This function performs a full table scan instead of using indexes because + * index scans could miss deleted tuples if an index has been re-indexed or + * re-created during change applications. While this approach may be slow on + * large tables, it is considered acceptable because it is only used in rare + * conflict cases where the target row for an update cannot be found. + */ +bool +FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot, + TransactionId *delete_xid, + RepOriginId *delete_origin, + TimestampTz *delete_time) +{ + TupleTableSlot *scanslot; + TableScanDesc scan; + TypeCacheEntry **eq; + TransactionId oldestXmin; + BufferHeapTupleTableSlot *hslot; + HeapTuple tuple; + Buffer buf; + Bitmapset *indexbitmap; + TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel); + + Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor)); + + *delete_xid = InvalidTransactionId; + *delete_origin = InvalidRepOriginId; + *delete_time = 0; + + /* Exit early if the commit timestamp data is not available */ + if (!track_commit_timestamp) + return false; + + /* Get the cutoff xmin for HeapTupleSatisfiesVacuum */ + oldestXmin = GetOldestNonRemovableTransactionId(rel); + + /* Get the index column bitmap for tuples_equal */ + indexbitmap = RelationGetIndexAttrBitmap(rel, + INDEX_ATTR_BITMAP_IDENTITY_KEY); + + /* fallback to PK if no replica identity */ + if (!indexbitmap) + indexbitmap = RelationGetIndexAttrBitmap(rel, + INDEX_ATTR_BITMAP_PRIMARY_KEY); + + eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts); + + /* + * Start a heap scan using SnapshotAny to identify dead tuples that are + * not visible under a standard MVCC snapshot. + */ + scan = table_beginscan(rel, SnapshotAny, 0, NULL); + scanslot = table_slot_create(rel, NULL); + hslot = (BufferHeapTupleTableSlot *) scanslot; + + table_rescan(scan, NULL); + + /* Try to find the tuple */ + while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot)) + { + bool dead = false; + TransactionId xmax; + TimestampTz localts; + RepOriginId localorigin; + + if (!tuples_equal(scanslot, searchslot, eq, indexbitmap)) + continue; + + tuple = ExecFetchSlotHeapTuple(scanslot, false, NULL); + buf = hslot->buffer; + + LockBuffer(buf, BUFFER_LOCK_SHARE); + + /* + * We do not consider HEAPTUPLE_DEAD status because it indicates + * either tuples whose inserting transaction was aborted, meaning + * there is no commit timestamp or origin, or tuples deleted by a + * transaction older than oldestXmin, making it safe to ignore them + * during conflict detection (See comments atop + * maybe_advance_nonremovable_xid() for details). + */ + if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD) + dead = true; + + LockBuffer(buf, BUFFER_LOCK_UNLOCK); + + if (!dead) + continue; + + xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data); + + /* Select the dead tuple with the most recent commit timestamp */ + if (TransactionIdGetCommitTsData(xmax, &localts, &localorigin) && + (TimestampDifferenceExceeds(*delete_time, localts, 0) || + *delete_time == 0)) + { + *delete_xid = xmax; + *delete_time = localts; + *delete_origin = localorigin; + } + } + + table_endscan(scan); + ExecDropSingleTupleTableSlot(scanslot); + + return *delete_time != 0; +} + /* * Find the tuple that violates the passed unique index (conflictindex). * diff --git a/src/backend/replication/logical/conflict.c b/src/backend/replication/logical/conflict.c index 97c4e26b5865..24d0b4ada4d0 100644 --- a/src/backend/replication/logical/conflict.c +++ b/src/backend/replication/logical/conflict.c @@ -25,6 +25,7 @@ static const char *const ConflictTypeNames[] = { [CT_INSERT_EXISTS] = "insert_exists", + [CT_UPDATE_DELETED] = "update_deleted", [CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs", [CT_UPDATE_EXISTS] = "update_exists", [CT_UPDATE_MISSING] = "update_missing", @@ -173,6 +174,7 @@ errcode_apply_conflict(ConflictType type) case CT_UPDATE_EXISTS: case CT_MULTIPLE_UNIQUE_CONFLICTS: return errcode(ERRCODE_UNIQUE_VIOLATION); + case CT_UPDATE_DELETED: case CT_UPDATE_ORIGIN_DIFFERS: case CT_UPDATE_MISSING: case CT_DELETE_ORIGIN_DIFFERS: @@ -246,6 +248,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo, break; + case CT_UPDATE_DELETED: + if (localts) + { + if (localorigin == InvalidRepOriginId) + appendStringInfo(&err_detail, _("The row to be updated was deleted locally in transaction %u at %s."), + localxmin, timestamptz_to_str(localts)); + else if (replorigin_by_oid(localorigin, true, &origin_name)) + appendStringInfo(&err_detail, _("The row to be updated was deleted by a different origin \"%s\" in transaction %u at %s."), + origin_name, localxmin, timestamptz_to_str(localts)); + + /* The origin that modified this row has been removed. */ + else + appendStringInfo(&err_detail, _("The row to be updated was deleted by a non-existent origin in transaction %u at %s."), + localxmin, timestamptz_to_str(localts)); + } + else + appendStringInfo(&err_detail, _("The row to be updated was deleted.")); + + break; + case CT_UPDATE_ORIGIN_DIFFERS: if (localorigin == InvalidRepOriginId) appendStringInfo(&err_detail, _("Updating the row that was modified locally in transaction %u at %s."), diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c index 6d595777a87c..d2205a84718d 100644 --- a/src/backend/replication/logical/worker.c +++ b/src/backend/replication/logical/worker.c @@ -2811,17 +2811,28 @@ apply_handle_update_internal(ApplyExecutionData *edata, } else { + ConflictType type; TupleTableSlot *newslot = localslot; + if (MySubscription->retainconflictinfo && + FindMostRecentlyDeletedTupleInfo(localrel, remoteslot, + &conflicttuple.xmin, + &conflicttuple.origin, + &conflicttuple.ts) && + conflicttuple.origin != replorigin_session_origin) + type = CT_UPDATE_DELETED; + else + type = CT_UPDATE_MISSING; + /* Store the new tuple for conflict reporting */ slot_store_data(newslot, relmapentry, newtup); /* - * The tuple to be updated could not be found. Do nothing except for - * emitting a log message. + * The tuple to be updated could not be found or was deleted. Do + * nothing except for emitting a log message. */ - ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_MISSING, - remoteslot, newslot, list_make1(&conflicttuple)); + ReportApplyConflict(estate, relinfo, LOG, type, remoteslot, newslot, + list_make1(&conflicttuple)); } /* Cleanup. */ @@ -3159,18 +3170,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata, remoteslot_part, &localslot); if (!found) { + ConflictType type; TupleTableSlot *newslot = localslot; + if (MySubscription->retainconflictinfo && + FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part, + &conflicttuple.xmin, + &conflicttuple.origin, + &conflicttuple.ts) && + conflicttuple.origin != replorigin_session_origin) + type = CT_UPDATE_DELETED; + else + type = CT_UPDATE_MISSING; + /* Store the new tuple for conflict reporting */ slot_store_data(newslot, part_entry, newtup); /* - * The tuple to be updated could not be found. Do nothing - * except for emitting a log message. + * The tuple to be updated could not be found or was + * deleted. Do nothing except for emitting a log message. */ ReportApplyConflict(estate, partrelinfo, LOG, - CT_UPDATE_MISSING, remoteslot_part, - newslot, list_make1(&conflicttuple)); + type, remoteslot_part, newslot, + list_make1(&conflicttuple)); return; } diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c index 97af7c6554ff..9f07b2a4b563 100644 --- a/src/backend/utils/adt/pgstatfuncs.c +++ b/src/backend/utils/adt/pgstatfuncs.c @@ -2171,7 +2171,7 @@ pg_stat_get_replication_slot(PG_FUNCTION_ARGS) Datum pg_stat_get_subscription_stats(PG_FUNCTION_ARGS) { -#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11 +#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 12 Oid subid = PG_GETARG_OID(0); TupleDesc tupdesc; Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0}; @@ -2193,19 +2193,21 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS) INT8OID, -1, 0); TupleDescInitEntry(tupdesc, (AttrNumber) 4, "confl_insert_exists", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 5, "confl_update_origin_differs", + TupleDescInitEntry(tupdesc, (AttrNumber) 5, "confl_update_deleted", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 6, "confl_update_exists", + TupleDescInitEntry(tupdesc, (AttrNumber) 6, "confl_update_origin_differs", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing", + TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_exists", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs", + TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_missing", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 9, "confl_delete_missing", + TupleDescInitEntry(tupdesc, (AttrNumber) 9, "confl_delete_origin_differs", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_multiple_unique_conflicts", + TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing", INT8OID, -1, 0); - TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset", + TupleDescInitEntry(tupdesc, (AttrNumber) 11, "confl_multiple_unique_conflicts", + INT8OID, -1, 0); + TupleDescInitEntry(tupdesc, (AttrNumber) 12, "stats_reset", TIMESTAMPTZOID, -1, 0); BlessTupleDesc(tupdesc); diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat index 2e7a337762fe..864a923bff19 100644 --- a/src/include/catalog/pg_proc.dat +++ b/src/include/catalog/pg_proc.dat @@ -5687,9 +5687,9 @@ { oid => '6231', descr => 'statistics: information about subscription stats', proname => 'pg_stat_get_subscription_stats', provolatile => 's', proparallel => 'r', prorettype => 'record', proargtypes => 'oid', - proallargtypes => '{oid,oid,int8,int8,int8,int8,int8,int8,int8,int8,int8,timestamptz}', - proargmodes => '{i,o,o,o,o,o,o,o,o,o,o,o}', - proargnames => '{subid,subid,apply_error_count,sync_error_count,confl_insert_exists,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,confl_multiple_unique_conflicts,stats_reset}', + proallargtypes => '{oid,oid,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,timestamptz}', + proargmodes => '{i,o,o,o,o,o,o,o,o,o,o,o,o}', + proargnames => '{subid,subid,apply_error_count,sync_error_count,confl_insert_exists,confl_update_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,confl_multiple_unique_conflicts,stats_reset}', prosrc => 'pg_stat_get_subscription_stats' }, { oid => '6118', descr => 'statistics: information about subscription', proname => 'pg_stat_get_subscription', prorows => '10', proisstrict => 'f', diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h index ae99407db89d..fcd2d1c23369 100644 --- a/src/include/executor/executor.h +++ b/src/include/executor/executor.h @@ -14,6 +14,7 @@ #ifndef EXECUTOR_H #define EXECUTOR_H +#include "datatype/timestamp.h" #include "executor/execdesc.h" #include "fmgr.h" #include "nodes/lockoptions.h" @@ -787,7 +788,11 @@ extern bool RelationFindReplTupleByIndex(Relation rel, Oid idxoid, TupleTableSlot *outslot); extern bool RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode, TupleTableSlot *searchslot, TupleTableSlot *outslot); - +extern bool FindMostRecentlyDeletedTupleInfo(Relation rel, + TupleTableSlot *searchslot, + TransactionId *delete_xid, + RepOriginId *delete_origin, + TimestampTz *delete_time); extern void ExecSimpleRelationInsert(ResultRelInfo *resultRelInfo, EState *estate, TupleTableSlot *slot); extern void ExecSimpleRelationUpdate(ResultRelInfo *resultRelInfo, diff --git a/src/include/replication/conflict.h b/src/include/replication/conflict.h index 6c59125f2565..cbd9656a60a5 100644 --- a/src/include/replication/conflict.h +++ b/src/include/replication/conflict.h @@ -26,6 +26,9 @@ typedef enum /* The row to be inserted violates unique constraint */ CT_INSERT_EXISTS, + /* The row to be updated was deleted by a different origin */ + CT_UPDATE_DELETED, + /* The row to be updated was modified by a different origin */ CT_UPDATE_ORIGIN_DIFFERS, diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out index 79ed5233edb6..d0428ac47fbf 100644 --- a/src/test/regress/expected/rules.out +++ b/src/test/regress/expected/rules.out @@ -2174,6 +2174,7 @@ pg_stat_subscription_stats| SELECT ss.subid, ss.apply_error_count, ss.sync_error_count, ss.confl_insert_exists, + ss.confl_update_deleted, ss.confl_update_origin_differs, ss.confl_update_exists, ss.confl_update_missing, @@ -2182,7 +2183,7 @@ pg_stat_subscription_stats| SELECT ss.subid, ss.confl_multiple_unique_conflicts, ss.stats_reset FROM pg_subscription s, - LATERAL pg_stat_get_subscription_stats(s.oid) ss(subid, apply_error_count, sync_error_count, confl_insert_exists, confl_update_origin_differs, confl_update_exists, confl_update_missing, confl_delete_origin_differs, confl_delete_missing, confl_multiple_unique_conflicts, stats_reset); + LATERAL pg_stat_get_subscription_stats(s.oid) ss(subid, apply_error_count, sync_error_count, confl_insert_exists, confl_update_deleted, confl_update_origin_differs, confl_update_exists, confl_update_missing, confl_delete_origin_differs, confl_delete_missing, confl_multiple_unique_conflicts, stats_reset); pg_stat_sys_indexes| SELECT relid, indexrelid, schemaname, diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl index 3bfe1631934e..4eacc0c089a6 100644 --- a/src/test/subscription/t/035_conflicts.pl +++ b/src/test/subscription/t/035_conflicts.pl @@ -149,7 +149,9 @@ # Setup a bidirectional logical replication between node_A & node_B ############################################################################### -# Initialize nodes. +# Initialize nodes. Enable the track_commit_timestamp on both nodes to detect +# the conflict when attempting to update a row that was previously modified by +# a different origin. # node_A. Increase the log_min_messages setting to DEBUG2 to debug test # failures. Disable autovacuum to avoid generating xid that could affect the @@ -157,7 +159,8 @@ my $node_A = $node_publisher; $node_A->append_conf( 'postgresql.conf', - qq{autovacuum = off + qq{track_commit_timestamp = on + autovacuum = off log_min_messages = 'debug2'}); $node_A->restart; @@ -227,6 +230,8 @@ ############################################################################### # Check that dead tuples on node A cannot be cleaned by VACUUM until the # concurrent transactions on Node B have been applied and flushed on Node A. +# And check that an update_deleted conflict is detected when updating a row +# that was deleted by a different origin. ############################################################################### # Insert a record @@ -255,10 +260,26 @@ qr/1 are dead but not yet removable/, 'the deleted column is non-removable'); +my $logfile = slurp_file($node_B->logfile(), $log_location); +ok( $logfile =~ + qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.* +.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .* +.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/, + 'delete target row was modified in tab'); + +$log_location = -s $node_A->logfile; + $node_A->safe_psql( 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;"); $node_B->wait_for_catchup($subname_AB); +$logfile = slurp_file($node_A->logfile(), $log_location); +ok( $logfile =~ + qr/conflict detected on relation "public.tab": conflict=update_deleted.* +.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .* +.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/, + 'update target row was deleted in tab'); + # Remember the next transaction ID to be assigned my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");